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 2013/11/21 23:41:23 UTC

[1/6] ACCUMULO-1009 moved ClientConfiguration into public API and removed its usage of AccumuloConfiguration

Updated Branches:
  refs/heads/master 9b4115582 -> 0d8027670


http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java b/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
index ff822ef..07aa1f8 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
@@ -36,7 +36,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.ServerClient;
-import org.apache.accumulo.core.client.impl.ServerConfigurationFactory;
+import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
 import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.client.impl.Translator;
@@ -145,7 +145,7 @@ public class BulkImporter {
           public void run() {
             List<TabletLocation> tabletsToAssignMapFileTo = Collections.emptyList();
             try {
-              tabletsToAssignMapFileTo = findOverlappingTablets(ServerConfigurationFactory.getConfiguration(instance), fs, locator, mapFile, credentials);
+              tabletsToAssignMapFileTo = findOverlappingTablets(ServerConfigurationUtil.getConfiguration(instance), fs, locator, mapFile, credentials);
             } catch (Exception ex) {
               log.warn("Unable to find tablets that overlap file " + mapFile.toString());
             }
@@ -208,7 +208,7 @@ public class BulkImporter {
             
             try {
               timer.start(Timers.QUERY_METADATA);
-              tabletsToAssignMapFileTo.addAll(findOverlappingTablets(ServerConfigurationFactory.getConfiguration(instance), fs, locator, entry.getKey(), ke, credentials));
+              tabletsToAssignMapFileTo.addAll(findOverlappingTablets(ServerConfigurationUtil.getConfiguration(instance), fs, locator, entry.getKey(), ke, credentials));
               timer.stop(Timers.QUERY_METADATA);
               keListIter.remove();
             } catch (Exception ex) {
@@ -585,8 +585,8 @@ public class BulkImporter {
   private List<KeyExtent> assignMapFiles(Credentials credentials, String location, Map<KeyExtent,List<PathSize>> assignmentsPerTablet)
       throws AccumuloException, AccumuloSecurityException {
     try {
-      long timeInMillis = ServerConfigurationFactory.getConfiguration(instance).getTimeInMillis(Property.TSERV_BULK_TIMEOUT);
-      TabletClientService.Iface client = ThriftUtil.getTServerClient(location, ServerConfigurationFactory.getConfiguration(instance), timeInMillis);
+      long timeInMillis = ServerConfigurationUtil.getConfiguration(instance).getTimeInMillis(Property.TSERV_BULK_TIMEOUT);
+      TabletClientService.Iface client = ThriftUtil.getTServerClient(location, ServerConfigurationUtil.getConfiguration(instance), timeInMillis);
       try {
         HashMap<KeyExtent,Map<String,org.apache.accumulo.core.data.thrift.MapFileInfo>> files = new HashMap<KeyExtent,Map<String,org.apache.accumulo.core.data.thrift.MapFileInfo>>();
         for (Entry<KeyExtent,List<PathSize>> entry : assignmentsPerTablet.entrySet()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/server/extras/src/main/java/org/apache/accumulo/utils/metanalysis/IndexMeta.java
----------------------------------------------------------------------
diff --git a/server/extras/src/main/java/org/apache/accumulo/utils/metanalysis/IndexMeta.java b/server/extras/src/main/java/org/apache/accumulo/utils/metanalysis/IndexMeta.java
index 5b85f18..b296f6d 100644
--- a/server/extras/src/main/java/org/apache/accumulo/utils/metanalysis/IndexMeta.java
+++ b/server/extras/src/main/java/org/apache/accumulo/utils/metanalysis/IndexMeta.java
@@ -23,10 +23,10 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.ColumnUpdate;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
----------------------------------------------------------------------
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
index 6534bdf..6afd42d 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
@@ -30,7 +30,7 @@ import java.util.UUID;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.impl.ServerConfigurationFactory;
+import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.gc.thrift.GCStatus;
 import org.apache.accumulo.core.gc.thrift.GcCycleStats;
@@ -134,7 +134,7 @@ public class GarbageCollectWriteAheadLogs {
   }
   
   private int removeFiles(Map<String,Path> nameToFileMap, Map<String,ArrayList<Path>> serverToFileMap, Map<String, Path> sortedWALogs, final GCStatus status) {
-    AccumuloConfiguration conf = ServerConfigurationFactory.getConfiguration(instance);
+    AccumuloConfiguration conf = ServerConfigurationUtil.getConfiguration(instance);
     for (Entry<String,ArrayList<Path>> entry : serverToFileMap.entrySet()) {
       if (entry.getKey().isEmpty()) {
         // old-style log entry, just remove it

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
----------------------------------------------------------------------
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
index 95a7262..1f1b28d 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
@@ -41,7 +41,7 @@ import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.ServerConfigurationFactory;
+import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -154,9 +154,9 @@ public class SimpleGarbageCollector implements Iface {
     this.credentials = credentials;
     this.instance = instance;
     
-    gcStartDelay = ServerConfigurationFactory.getConfiguration(instance).getTimeInMillis(Property.GC_CYCLE_START);
-    long gcDelay = ServerConfigurationFactory.getConfiguration(instance).getTimeInMillis(Property.GC_CYCLE_DELAY);
-    numDeleteThreads = ServerConfigurationFactory.getConfiguration(instance).getCount(Property.GC_DELETE_THREADS);
+    gcStartDelay = ServerConfigurationUtil.getConfiguration(instance).getTimeInMillis(Property.GC_CYCLE_START);
+    long gcDelay = ServerConfigurationUtil.getConfiguration(instance).getTimeInMillis(Property.GC_CYCLE_DELAY);
+    numDeleteThreads = ServerConfigurationUtil.getConfiguration(instance).getCount(Property.GC_DELETE_THREADS);
     log.info("start delay: " + gcStartDelay + " milliseconds");
     log.info("time delay: " + gcDelay + " milliseconds");
     log.info("safemode: " + opts.safeMode);
@@ -482,7 +482,7 @@ public class SimpleGarbageCollector implements Iface {
       
       Trace.offNoFlush();
       try {
-        long gcDelay = ServerConfigurationFactory.getConfiguration(instance).getTimeInMillis(Property.GC_CYCLE_DELAY);
+        long gcDelay = ServerConfigurationUtil.getConfiguration(instance).getTimeInMillis(Property.GC_CYCLE_DELAY);
         log.debug("Sleeping for " + gcDelay + " milliseconds");
         Thread.sleep(gcDelay);
       } catch (InterruptedException e) {
@@ -535,8 +535,8 @@ public class SimpleGarbageCollector implements Iface {
   
   private HostAndPort startStatsService() throws UnknownHostException {
     Processor<Iface> processor = new Processor<Iface>(TraceWrap.service(this));
-    int port = ServerConfigurationFactory.getConfiguration(instance).getPort(Property.GC_PORT);
-    long maxMessageSize = ServerConfigurationFactory.getConfiguration(instance).getMemoryInBytes(Property.GENERAL_MAX_MESSAGE_SIZE);
+    int port = ServerConfigurationUtil.getConfiguration(instance).getPort(Property.GC_PORT);
+    long maxMessageSize = ServerConfigurationUtil.getConfiguration(instance).getMemoryInBytes(Property.GENERAL_MAX_MESSAGE_SIZE);
     HostAndPort result = HostAndPort.fromParts(opts.getAddress(), port);
     try {
       port = TServerUtils.startTServer(result, processor, this.getClass().getSimpleName(), "GC Monitor Service", 2, 1000, maxMessageSize).address.getPort();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/server/master/src/main/java/org/apache/accumulo/master/Master.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/Master.java b/server/master/src/main/java/org/apache/accumulo/master/Master.java
index c6f5ebb..e17bccc 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/Master.java
@@ -44,7 +44,7 @@ import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TableOperationsImpl;
-import org.apache.accumulo.core.client.impl.ServerConfigurationFactory;
+import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.ThriftTransportPool;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
@@ -787,7 +787,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
 
     private void updatePlugins(String property) {
       if (property.equals(Property.MASTER_TABLET_BALANCER.getKey())) {
-        TabletBalancer balancer = ServerConfigurationFactory.getConfiguration(instance).instantiateClassProperty(Property.MASTER_TABLET_BALANCER, TabletBalancer.class,
+        TabletBalancer balancer = ServerConfigurationUtil.getConfiguration(instance).instantiateClassProperty(Property.MASTER_TABLET_BALANCER, TabletBalancer.class,
             new DefaultLoadBalancer());
         balancer.init(serverConfig);
         tabletBalancer = balancer;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/test/src/main/java/org/apache/accumulo/test/IMMLGBenchmark.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/IMMLGBenchmark.java b/test/src/main/java/org/apache/accumulo/test/IMMLGBenchmark.java
index 7bee351..8fdba5a 100644
--- a/test/src/main/java/org/apache/accumulo/test/IMMLGBenchmark.java
+++ b/test/src/main/java/org/apache/accumulo/test/IMMLGBenchmark.java
@@ -30,13 +30,13 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/test/src/main/java/org/apache/accumulo/test/performance/metadata/MetadataBatchScanTest.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/metadata/MetadataBatchScanTest.java b/test/src/main/java/org/apache/accumulo/test/performance/metadata/MetadataBatchScanTest.java
index cf4f134..1a314bf 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/metadata/MetadataBatchScanTest.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/metadata/MetadataBatchScanTest.java
@@ -27,10 +27,10 @@ import java.util.UUID;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
index 05384d7..a26e69e 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
@@ -24,12 +24,12 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.accumulo.core.cli.Help;
+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.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.KeyExtent;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/test/src/main/java/org/apache/accumulo/test/randomwalk/State.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/State.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/State.java
index 5227b2a..dc6e972 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/State.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/State.java
@@ -25,13 +25,13 @@ import java.util.concurrent.TimeUnit;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.MultiTableBatchWriter;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.log4j.Logger;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java
index 749209e..d92dea2 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/CopyTool.java
@@ -18,10 +18,10 @@ package org.apache.accumulo.test.randomwalk.multitable;
 
 import java.io.IOException;
 
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/test/src/main/java/org/apache/accumulo/test/randomwalk/sequential/MapRedVerifyTool.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/sequential/MapRedVerifyTool.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/sequential/MapRedVerifyTool.java
index b0c5029..22a7371 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/sequential/MapRedVerifyTool.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/sequential/MapRedVerifyTool.java
@@ -19,10 +19,10 @@ package org.apache.accumulo.test.randomwalk.sequential;
 import java.io.IOException;
 import java.util.Iterator;
 
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/test/src/main/java/org/apache/accumulo/test/scalability/ScaleTest.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/scalability/ScaleTest.java b/test/src/main/java/org/apache/accumulo/test/scalability/ScaleTest.java
index c20d004..c4dd42d 100644
--- a/test/src/main/java/org/apache/accumulo/test/scalability/ScaleTest.java
+++ b/test/src/main/java/org/apache/accumulo/test/scalability/ScaleTest.java
@@ -21,10 +21,10 @@ import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.hadoop.io.Text;
 
 public abstract class ScaleTest {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/test/src/test/java/org/apache/accumulo/test/MultiTableBatchWriterTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/MultiTableBatchWriterTest.java b/test/src/test/java/org/apache/accumulo/test/MultiTableBatchWriterTest.java
index ace5d24..5c2698e 100644
--- a/test/src/test/java/org/apache/accumulo/test/MultiTableBatchWriterTest.java
+++ b/test/src/test/java/org/apache/accumulo/test/MultiTableBatchWriterTest.java
@@ -24,6 +24,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.MultiTableBatchWriter;
 import org.apache.accumulo.core.client.MutationsRejectedException;
@@ -34,7 +35,6 @@ import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.impl.MultiTableBatchWriterImpl;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;


[2/6] git commit: ACCUMULO-1009 moved ClientConfiguration into public API and removed its usage of AccumuloConfiguration

Posted by ct...@apache.org.
ACCUMULO-1009 moved ClientConfiguration into public API and removed its usage of AccumuloConfiguration


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

Branch: refs/heads/master
Commit: 5bd68ef9b751848a441cb56ca82a7e2aafdf1461
Parents: fe46a60
Author: Keith Turner <kt...@apache.org>
Authored: Thu Nov 21 15:03:03 2013 -0500
Committer: Keith Turner <kt...@apache.org>
Committed: Thu Nov 21 15:04:53 2013 -0500

----------------------------------------------------------------------
 .../apache/accumulo/core/cli/ClientOpts.java    |   4 +-
 .../core/client/ClientConfiguration.java        | 260 ++++++++++++++++
 .../accumulo/core/client/ZooKeeperInstance.java |  19 +-
 .../client/admin/InstanceOperationsImpl.java    |   8 +-
 .../core/client/admin/TableOperationsImpl.java  |   6 +-
 .../core/client/impl/ConditionalWriterImpl.java |   6 +-
 .../accumulo/core/client/impl/MasterClient.java |   2 +-
 .../core/client/impl/OfflineScanner.java        |   4 +-
 .../core/client/impl/ScannerIterator.java       |   2 +-
 .../accumulo/core/client/impl/ServerClient.java |   4 +-
 .../client/impl/ServerConfigurationFactory.java |  30 --
 .../client/impl/ServerConfigurationUtil.java    |  63 ++++
 .../impl/TabletServerBatchReaderIterator.java   |   2 +-
 .../client/impl/TabletServerBatchWriter.java    |   6 +-
 .../accumulo/core/client/impl/Writer.java       |   2 +-
 .../core/client/mapred/AbstractInputFormat.java |   2 +-
 .../core/client/mapred/AccumuloInputFormat.java |   2 +-
 .../mapred/AccumuloMultiTableInputFormat.java   |   2 +-
 .../client/mapred/AccumuloOutputFormat.java     |   2 +-
 .../client/mapred/AccumuloRowInputFormat.java   |   2 +-
 .../client/mapreduce/AbstractInputFormat.java   |   2 +-
 .../client/mapreduce/AccumuloInputFormat.java   |   2 +-
 .../AccumuloMultiTableInputFormat.java          |   2 +-
 .../client/mapreduce/AccumuloOutputFormat.java  |   2 +-
 .../mapreduce/AccumuloRowInputFormat.java       |   2 +-
 .../mapreduce/lib/util/ConfiguratorBase.java    |   2 +-
 .../accumulo/core/conf/ClientConfiguration.java | 310 -------------------
 .../core/metadata/MetadataLocationObtainer.java |   8 +-
 .../apache/accumulo/core/util/shell/Shell.java  |  11 +-
 .../core/util/shell/ShellOptionsJC.java         |   4 +-
 .../lib/util/ConfiguratorBaseTest.java          |   4 +-
 .../core/conf/ClientConfigurationTest.java      |   3 +-
 .../core/util/shell/ShellSetInstanceTest.java   |  17 +-
 .../examples/simple/filedata/FileDataQuery.java |   2 +-
 .../simple/mapreduce/TokenFileWordCount.java    |   2 +-
 .../examples/simple/reservations/ARS.java       |   2 +-
 .../minicluster/MiniAccumuloCluster.java        |   2 +-
 .../minicluster/MiniAccumuloInstance.java       |   2 +-
 .../org/apache/accumulo/proxy/ProxyServer.java  |   2 +-
 .../accumulo/server/client/BulkImporter.java    |  10 +-
 .../accumulo/utils/metanalysis/IndexMeta.java   |   2 +-
 .../gc/GarbageCollectWriteAheadLogs.java        |   4 +-
 .../accumulo/gc/SimpleGarbageCollector.java     |  14 +-
 .../java/org/apache/accumulo/master/Master.java |   4 +-
 .../apache/accumulo/test/IMMLGBenchmark.java    |   2 +-
 .../metadata/MetadataBatchScanTest.java         |   2 +-
 .../test/performance/thrift/NullTserver.java    |   2 +-
 .../apache/accumulo/test/randomwalk/State.java  |   2 +-
 .../test/randomwalk/multitable/CopyTool.java    |   2 +-
 .../randomwalk/sequential/MapRedVerifyTool.java |   2 +-
 .../accumulo/test/scalability/ScaleTest.java    |   2 +-
 .../test/MultiTableBatchWriterTest.java         |   2 +-
 52 files changed, 422 insertions(+), 437 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java b/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
index 1d26a00..c43b121 100644
--- a/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
+++ b/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
@@ -24,9 +24,11 @@ import java.util.UUID;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
@@ -35,8 +37,6 @@ import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.Properties;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ClientConfiguration;
-import org.apache.accumulo.core.conf.ClientConfiguration.ClientProperty;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.security.Authorizations;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/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
new file mode 100644
index 0000000..aa1c6fc
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
@@ -0,0 +1,260 @@
+/*
+ * 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;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.StringReader;
+import java.io.StringWriter;
+import java.util.Arrays;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.conf.PropertyType;
+import org.apache.accumulo.core.util.ArgumentChecker;
+import org.apache.commons.configuration.CompositeConfiguration;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
+
+/**
+ * Contains a list of property keys recognized by the Accumulo client and convenience methods for setting them.
+ * 
+ * @since 1.6.0
+ */
+public class ClientConfiguration extends CompositeConfiguration {
+  public static final String USER_ACCUMULO_DIR_NAME = ".accumulo";
+  public static final String USER_CONF_FILENAME = "config";
+  public static final String GLOBAL_CONF_FILENAME = "client.conf";
+
+  public enum ClientProperty {
+    RPC_SSL_TRUSTSTORE_PATH(Property.RPC_SSL_TRUSTSTORE_PATH),
+    RPC_SSL_TRUSTSTORE_PASSWORD(Property.RPC_SSL_TRUSTSTORE_PASSWORD),
+    RPC_SSL_TRUSTSTORE_TYPE(Property.RPC_SSL_TRUSTSTORE_TYPE),
+    RPC_SSL_KEYSTORE_PATH(Property.RPC_SSL_KEYSTORE_PATH),
+    RPC_SSL_KEYSTORE_PASSWORD(Property.RPC_SSL_KEYSTORE_PASSWORD),
+    RPC_SSL_KEYSTORE_TYPE(Property.RPC_SSL_KEYSTORE_TYPE),
+    RPC_USE_JSSE(Property.RPC_USE_JSSE),
+    INSTANCE_RPC_SSL_CLIENT_AUTH(Property.INSTANCE_RPC_SSL_CLIENT_AUTH),
+    INSTANCE_RPC_SSL_ENABLED(Property.INSTANCE_RPC_SSL_ENABLED),
+    INSTANCE_ZK_HOST(Property.INSTANCE_ZK_HOST),
+    INSTANCE_ZK_TIMEOUT(Property.INSTANCE_ZK_TIMEOUT),
+    INSTANCE_NAME("client.instance.name", null, PropertyType.STRING, "Name of Accumulo instance to connect to"),
+    INSTANCE_ID("client.instance.id", null, PropertyType.STRING, "UUID of Accumulo instance to connect to"),
+    ;
+
+    private String key;
+    private String defaultValue;
+    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) {
+      this.key = key;
+      this.defaultValue = defaultValue;
+      this.type = type;
+      this.description = description;
+    }
+
+    public String getKey() {
+      return key;
+    }
+
+    public String getDefaultValue() {
+      return defaultValue;
+    }
+
+    public PropertyType getType() {
+      return type;
+    }
+
+    public String getDescription() {
+      return description;
+    }
+
+    public Property getAccumuloProperty() {
+      return accumuloProperty;
+    }
+
+    public static ClientProperty getPropertyByKey(String key) {
+      for (ClientProperty prop : ClientProperty.values())
+        if (prop.getKey().equals(key))
+          return prop;
+      return null;
+    }
+  };
+
+  public ClientConfiguration(List<? extends Configuration> configs) {
+    super(configs);
+  }
+
+  public ClientConfiguration(Configuration... configs) {
+    this(Arrays.asList(configs));
+  }
+
+  public static ClientConfiguration loadDefault() {
+    return loadFromSearchPath(getDefaultSearchPath());
+  }
+
+  public static ClientConfiguration loadDefault(String overridePropertiesFilename) throws FileNotFoundException, ConfigurationException {
+    if (overridePropertiesFilename == null)
+      return loadDefault();
+    else
+      return new ClientConfiguration(new PropertiesConfiguration(overridePropertiesFilename));
+  }
+
+  private static ClientConfiguration loadFromSearchPath(List<String> paths) {
+    try {
+      List<Configuration> configs = new LinkedList<Configuration>();
+      for (String path : paths) {
+        File conf = new File(path);
+        if (conf.canRead()) {
+          configs.add(new PropertiesConfiguration(conf));
+       }
+      }
+      return new ClientConfiguration(configs);
+    } catch (ConfigurationException e) {
+      throw new IllegalStateException("Error loading client configuration", e);
+    }
+  }
+
+  public static ClientConfiguration deserialize(String serializedConfig) {
+    PropertiesConfiguration propConfig = new PropertiesConfiguration();
+    try {
+      propConfig.load(new StringReader(serializedConfig));
+    } catch (ConfigurationException e) {
+      throw new IllegalArgumentException("Error deserializing client configuration: " + serializedConfig, e);
+    }
+    return new ClientConfiguration(propConfig);
+  }
+
+  private static List<String> getDefaultSearchPath() {
+    String clientConfSearchPath = System.getenv("ACCUMULO_CLIENT_CONF_PATH");
+    List<String> clientConfPaths;
+    if (clientConfSearchPath != null) {
+      clientConfPaths = Arrays.asList(clientConfSearchPath.split(File.pathSeparator));
+    } else {
+      // if $ACCUMULO_CLIENT_CONF_PATH env isn't set, priority from top to bottom is:
+      // ~/.accumulo/config
+      // $ACCUMULO_CONF_DIR/client.conf -OR- $ACCUMULO_HOME/conf/client.conf (depending on whether $ACCUMULO_CONF_DIR is set)
+      // /etc/accumulo/client.conf
+      clientConfPaths = new LinkedList<String>();
+      clientConfPaths.add(System.getProperty("user.home") + File.separator + USER_ACCUMULO_DIR_NAME + File.separator + USER_CONF_FILENAME);
+      if (System.getenv("ACCUMULO_CONF_DIR") != null) {
+        clientConfPaths.add(System.getenv("ACCUMULO_CONF_DIR") + File.separator + GLOBAL_CONF_FILENAME);
+      } else if (System.getenv("ACCUMULO_HOME") != null) {
+        clientConfPaths.add(System.getenv("ACCUMULO_HOME") + File.separator + "conf" + File.separator + GLOBAL_CONF_FILENAME);
+      }
+      clientConfPaths.add("/etc/accumulo/" + GLOBAL_CONF_FILENAME);
+    }
+    return clientConfPaths;
+  }
+
+  public String serialize() {
+    PropertiesConfiguration propConfig = new PropertiesConfiguration();
+    propConfig.copy(this);
+    StringWriter writer = new StringWriter();
+    try {
+      propConfig.save(writer);
+    } catch (ConfigurationException e) {
+      // this should never happen
+      throw new IllegalStateException(e);
+    }
+    return writer.toString();
+  }
+
+  public String get(ClientProperty prop) {
+    if (this.containsKey(prop.getKey()))
+      return this.getString(prop.getKey());
+    else
+      return prop.getDefaultValue();
+  }
+
+  public void setProperty(ClientProperty prop, String value) {
+    this.setProperty(prop.getKey(), value);
+  }
+
+  public ClientConfiguration with(ClientProperty prop, String value) {
+    this.setProperty(prop.getKey(), value);
+    return this;
+  }
+
+  public ClientConfiguration withInstance(String instanceName) {
+    ArgumentChecker.notNull(instanceName);
+    return with(ClientProperty.INSTANCE_NAME, instanceName);
+  }
+
+  public ClientConfiguration withInstance(UUID instanceId) {
+    ArgumentChecker.notNull(instanceId);
+    return with(ClientProperty.INSTANCE_ID, instanceId.toString());
+  }
+
+  public ClientConfiguration withZkHosts(String zooKeepers) {
+    ArgumentChecker.notNull(zooKeepers);
+    return with(ClientProperty.INSTANCE_ZK_HOST, zooKeepers);
+  }
+
+  public ClientConfiguration withZkTimeout(int timeout) {
+    return with(ClientProperty.INSTANCE_ZK_TIMEOUT, String.valueOf(timeout));
+  }
+
+  public ClientConfiguration withSsl(boolean sslEnabled) {
+    return withSsl(sslEnabled, false);
+  }
+
+  public ClientConfiguration withSsl(boolean sslEnabled, boolean useJsseConfig) {
+    return with(ClientProperty.INSTANCE_RPC_SSL_ENABLED, String.valueOf(sslEnabled))
+        .with(ClientProperty.RPC_USE_JSSE, String.valueOf(useJsseConfig));
+  }
+
+  public ClientConfiguration withTruststore(String path) {
+    return withTruststore(path, null, null);
+  }
+
+  public ClientConfiguration withTruststore(String path, String password, String type) {
+    ArgumentChecker.notNull(path);
+    setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_PATH, path);
+    if (password != null)
+      setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_PASSWORD, password);
+    if (type != null)
+      setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_TYPE, type);
+    return this;
+  }
+
+  public ClientConfiguration withKeystore(String path) {
+    return withKeystore(path, null, null);
+  }
+
+  public ClientConfiguration withKeystore(String path, String password, String type) {
+    ArgumentChecker.notNull(path);
+    setProperty(ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH, "true");
+    setProperty(ClientProperty.RPC_SSL_KEYSTORE_PATH, path);
+    if (password != null)
+      setProperty(ClientProperty.RPC_SSL_KEYSTORE_PASSWORD, password);
+    if (type != null)
+      setProperty(ClientProperty.RPC_SSL_KEYSTORE_TYPE, type);
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/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 fb4ab79..caf6864 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
@@ -23,12 +23,13 @@ import java.util.UUID;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 import org.apache.accumulo.core.client.impl.ConnectorImpl;
+import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
 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.ClientConfiguration;
-import org.apache.accumulo.core.conf.ClientConfiguration.ClientProperty;
+import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.ArgumentChecker;
@@ -81,7 +82,7 @@ public class ZooKeeperInstance implements Instance {
    *          The name of specific accumulo instance. This is set at initialization time.
    * @param zooKeepers
    *          A comma separated list of zoo keeper server locations. Each location can contain an optional port, of the format host:port.
-   * @deprecated since 1.6.0; Use {@link #ZooKeeperInstance(ClientConfiguration)} instead.
+   * @deprecated since 1.6.0; Use {@link #ZooKeeperInstance(Configuration)} instead.
    */
   @Deprecated
   public ZooKeeperInstance(String instanceName, String zooKeepers) {
@@ -96,7 +97,7 @@ public class ZooKeeperInstance implements Instance {
    *          A comma separated list of zoo keeper server locations. Each location can contain an optional port, of the format host:port.
    * @param sessionTimeout
    *          zoo keeper session time out in milliseconds.
-   * @deprecated since 1.6.0; Use {@link #ZooKeeperInstance(ClientConfiguration)} instead.
+   * @deprecated since 1.6.0; Use {@link #ZooKeeperInstance(Configuration)} instead.
    */
   @Deprecated
   public ZooKeeperInstance(String instanceName, String zooKeepers, int sessionTimeout) {
@@ -109,7 +110,7 @@ public class ZooKeeperInstance implements Instance {
    *          The UUID that identifies the accumulo instance you want to connect to.
    * @param zooKeepers
    *          A comma separated list of zoo keeper server locations. Each location can contain an optional port, of the format host:port.
-   * @deprecated since 1.6.0; Use {@link #ZooKeeperInstance(ClientConfiguration)} instead.
+   * @deprecated since 1.6.0; Use {@link #ZooKeeperInstance(Configuration)} instead.
    */
   @Deprecated
   public ZooKeeperInstance(UUID instanceId, String zooKeepers) {
@@ -124,7 +125,7 @@ public class ZooKeeperInstance implements Instance {
    *          A comma separated list of zoo keeper server locations. Each location can contain an optional port, of the format host:port.
    * @param sessionTimeout
    *          zoo keeper session time out in milliseconds.
-   * @deprecated since 1.6.0; Use {@link #ZooKeeperInstance(ClientConfiguration)} instead.
+   * @deprecated since 1.6.0; Use {@link #ZooKeeperInstance(Configuration)} instead.
    */
   @Deprecated
   public ZooKeeperInstance(UUID instanceId, String zooKeepers, int sessionTimeout) {
@@ -264,11 +265,9 @@ public class ZooKeeperInstance implements Instance {
   }
 
   @Override
+  @Deprecated
   public AccumuloConfiguration getConfiguration() {
-    if (accumuloConf == null) {
-      accumuloConf = clientConf.getAccumuloConfiguration();
-    }
-    return accumuloConf;
+    return ServerConfigurationUtil.convertClientConfig(accumuloConf == null ? DefaultConfiguration.getInstance() : accumuloConf, clientConf);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
index 333201e..85bc1a3 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
@@ -30,7 +30,7 @@ import org.apache.accumulo.core.client.impl.ClientExec;
 import org.apache.accumulo.core.client.impl.ClientExecReturn;
 import org.apache.accumulo.core.client.impl.MasterClient;
 import org.apache.accumulo.core.client.impl.ServerClient;
-import org.apache.accumulo.core.client.impl.ServerConfigurationFactory;
+import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
 import org.apache.accumulo.core.client.impl.thrift.ConfigurationType;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
@@ -132,7 +132,7 @@ public class InstanceOperationsImpl implements InstanceOperations {
   public List<ActiveScan> getActiveScans(String tserver) throws AccumuloException, AccumuloSecurityException {
     Client client = null;
     try {
-      client = ThriftUtil.getTServerClient(tserver, ServerConfigurationFactory.getConfiguration(instance));
+      client = ThriftUtil.getTServerClient(tserver, ServerConfigurationUtil.getConfiguration(instance));
       
       List<ActiveScan> as = new ArrayList<ActiveScan>();
       for (org.apache.accumulo.core.tabletserver.thrift.ActiveScan activeScan : client.getActiveScans(Tracer.traceInfo(), credentials.toThrift(instance))) {
@@ -169,7 +169,7 @@ public class InstanceOperationsImpl implements InstanceOperations {
   public List<ActiveCompaction> getActiveCompactions(String tserver) throws AccumuloException, AccumuloSecurityException {
     Client client = null;
     try {
-      client = ThriftUtil.getTServerClient(tserver, ServerConfigurationFactory.getConfiguration(instance));
+      client = ThriftUtil.getTServerClient(tserver, ServerConfigurationUtil.getConfiguration(instance));
       
       List<ActiveCompaction> as = new ArrayList<ActiveCompaction>();
       for (org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction activeCompaction : client.getActiveCompactions(Tracer.traceInfo(),
@@ -193,7 +193,7 @@ public class InstanceOperationsImpl implements InstanceOperations {
   public void ping(String tserver) throws AccumuloException {
     TTransport transport = null;
     try {
-      transport = ThriftUtil.createTransport(AddressUtil.parseAddress(tserver), ServerConfigurationFactory.getConfiguration(instance));
+      transport = ThriftUtil.createTransport(AddressUtil.parseAddress(tserver), ServerConfigurationUtil.getConfiguration(instance));
       TabletClientService.Client client = ThriftUtil.createClient(new TabletClientService.Client.Factory(), transport);
       client.getTabletServerStatus(Tracer.traceInfo(), credentials.toThrift(instance));
     } catch (TTransportException e) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
index a85772d..a779ae4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
@@ -61,7 +61,7 @@ import org.apache.accumulo.core.client.impl.ClientExec;
 import org.apache.accumulo.core.client.impl.ClientExecReturn;
 import org.apache.accumulo.core.client.impl.MasterClient;
 import org.apache.accumulo.core.client.impl.ServerClient;
-import org.apache.accumulo.core.client.impl.ServerConfigurationFactory;
+import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
@@ -470,7 +470,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
         }
         
         try {
-          TabletClientService.Client client = ThriftUtil.getTServerClient(tl.tablet_location, ServerConfigurationFactory.getConfiguration(instance));
+          TabletClientService.Client client = ThriftUtil.getTServerClient(tl.tablet_location, ServerConfigurationUtil.getConfiguration(instance));
           try {
             OpTimer opTimer = null;
             if (log.isTraceEnabled())
@@ -1113,7 +1113,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       ret = new Path(dir);
       fs = ret.getFileSystem(CachedConfiguration.getInstance());
     } else {
-      fs = FileUtil.getFileSystem(CachedConfiguration.getInstance(), ServerConfigurationFactory.getConfiguration(instance));
+      fs = FileUtil.getFileSystem(CachedConfiguration.getInstance(), ServerConfigurationUtil.getConfiguration(instance));
       ret = fs.makeQualified(new Path(dir));
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
index 1d35af4..cd89adb 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
@@ -536,10 +536,10 @@ class ConditionalWriterImpl implements ConditionalWriter {
   
   private TabletClientService.Iface getClient(String location) throws TTransportException {
     TabletClientService.Iface client;
-    if (timeout < ServerConfigurationFactory.getConfiguration(instance).getTimeInMillis(Property.GENERAL_RPC_TIMEOUT))
-      client = ThriftUtil.getTServerClient(location, ServerConfigurationFactory.getConfiguration(instance), timeout);
+    if (timeout < ServerConfigurationUtil.getConfiguration(instance).getTimeInMillis(Property.GENERAL_RPC_TIMEOUT))
+      client = ThriftUtil.getTServerClient(location, ServerConfigurationUtil.getConfiguration(instance), timeout);
     else
-      client = ThriftUtil.getTServerClient(location, ServerConfigurationFactory.getConfiguration(instance));
+      client = ThriftUtil.getTServerClient(location, ServerConfigurationUtil.getConfiguration(instance));
     return client;
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java
index 4cf6e06..6bef3a7 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java
@@ -61,7 +61,7 @@ public class MasterClient {
     
     try {
       // Master requests can take a long time: don't ever time out
-      MasterClientService.Client client = ThriftUtil.getClientNoTimeout(new MasterClientService.Client.Factory(), master, ServerConfigurationFactory.getConfiguration(instance));
+      MasterClientService.Client client = ThriftUtil.getClientNoTimeout(new MasterClientService.Client.Factory(), master, ServerConfigurationUtil.getConfiguration(instance));
       return client;
     } catch (TTransportException tte) {
       if (tte.getCause().getClass().equals(UnknownHostException.class)) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java b/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
index 5e92d8b..6e08710 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
@@ -227,7 +227,7 @@ class OfflineIterator implements Iterator<Entry<Key,Value>> {
     if (currentExtent != null && !extent.isPreviousExtent(currentExtent))
       throw new AccumuloException(" " + currentExtent + " is not previous extent " + extent);
     
-    String tablesDir = ServerConfigurationFactory.getConfiguration(instance).get(Property.INSTANCE_DFS_DIR) + "/tables";
+    String tablesDir = ServerConfigurationUtil.getConfiguration(instance).get(Property.INSTANCE_DFS_DIR) + "/tables";
 
     List<String> absFiles = new ArrayList<String>();
     for (String relPath : relFiles) {
@@ -296,7 +296,7 @@ class OfflineIterator implements Iterator<Entry<Key,Value>> {
     
     Configuration conf = CachedConfiguration.getInstance();
     
-    FileSystem defaultFs = FileUtil.getFileSystem(conf, ServerConfigurationFactory.getConfiguration(instance));
+    FileSystem defaultFs = FileUtil.getFileSystem(conf, ServerConfigurationUtil.getConfiguration(instance));
     
     for (SortedKeyValueIterator<Key,Value> reader : readers) {
       ((FileSKVIterator) reader).close();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
index 677a751..5ea3662 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
@@ -81,7 +81,7 @@ public class ScannerIterator implements Iterator<Entry<Key,Value>> {
       
       try {
         while (true) {
-          List<KeyValue> currentBatch = ThriftScanner.scan(instance, credentials, scanState, timeOut, ServerConfigurationFactory.getConfiguration(instance));
+          List<KeyValue> currentBatch = ThriftScanner.scan(instance, credentials, scanState, timeOut, ServerConfigurationUtil.getConfiguration(instance));
           
           if (currentBatch == null) {
             synchQ.add(EMPTY_LIST);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
index 89956db..4eb845d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
@@ -125,7 +125,7 @@ public class ServerClient {
   }
   
   public static Pair<String,ClientService.Client> getConnection(Instance instance, boolean preferCachedConnections) throws TTransportException {
-    AccumuloConfiguration conf = ServerConfigurationFactory.getConfiguration(instance);
+    AccumuloConfiguration conf = ServerConfigurationUtil.getConfiguration(instance);
     return getConnection(instance, preferCachedConnections, conf.getTimeInMillis(Property.GENERAL_RPC_TIMEOUT));
   }
   
@@ -142,7 +142,7 @@ public class ServerClient {
       if (data != null && !new String(data).equals("master"))
         servers.add(new ThriftTransportKey(
           new ServerServices(new String(data)).getAddressString(Service.TSERV_CLIENT),
-          rpcTimeout, SslConnectionParams.forClient(ServerConfigurationFactory.getConfiguration(instance))));
+          rpcTimeout, SslConnectionParams.forClient(ServerConfigurationUtil.getConfiguration(instance))));
     }
     
     boolean opened = false;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/impl/ServerConfigurationFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerConfigurationFactory.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerConfigurationFactory.java
deleted file mode 100644
index 2c3427a..0000000
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerConfigurationFactory.java
+++ /dev/null
@@ -1,30 +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.impl;
-
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-
-/**
- * All client side code that needs a server side configuration object should obtain it from here.
- */
-public class ServerConfigurationFactory {
-  @SuppressWarnings("deprecation")
-  public static AccumuloConfiguration getConfiguration(Instance instance) {
-    return instance.getConfiguration();
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/impl/ServerConfigurationUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerConfigurationUtil.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerConfigurationUtil.java
new file mode 100644
index 0000000..8021f76
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerConfigurationUtil.java
@@ -0,0 +1,63 @@
+/*
+ * 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.impl;
+
+import java.util.Iterator;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.commons.configuration.Configuration;
+
+/**
+ * All client side code that needs a server side configuration object should obtain it from here.
+ */
+public class ServerConfigurationUtil {
+  @SuppressWarnings("deprecation")
+  public static AccumuloConfiguration getConfiguration(Instance instance) {
+    return instance.getConfiguration();
+  }
+  
+  public static AccumuloConfiguration convertClientConfig(final AccumuloConfiguration base, final Configuration config) {
+
+    return new AccumuloConfiguration() {
+      @Override
+      public String get(Property property) {
+        if (config.containsKey(property.getKey()))
+          return config.getString(property.getKey());
+        else
+          return base.get(property);
+      }
+
+      @Override
+      public void getProperties(Map<String,String> props, PropertyFilter filter) {
+
+        base.getProperties(props, filter);
+
+        @SuppressWarnings("unchecked")
+        Iterator<String> keyIter = config.getKeys();
+        while (keyIter.hasNext()) {
+          String key = keyIter.next();
+          if (filter.accept(key))
+            props.put(key, config.getString(key));
+        }
+      }
+    };
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
index 9961f8f..d82056b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
@@ -362,7 +362,7 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
           timeoutTrackers.put(tsLocation, timeoutTracker);
         }
         doLookup(instance, credentials, tsLocation, tabletsRanges, tsFailures, unscanned, receiver, columns, options, authorizations,
-            ServerConfigurationFactory.getConfiguration(instance), timeoutTracker);
+            ServerConfigurationUtil.getConfiguration(instance), timeoutTracker);
         if (tsFailures.size() > 0) {
           locator.invalidateCache(tsFailures.keySet());
           synchronized (failures) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
index b79ae39..8a51657 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
@@ -855,10 +855,10 @@ public class TabletServerBatchWriter {
       try {
         TabletClientService.Iface client;
         
-        if (timeoutTracker.getTimeOut() < ServerConfigurationFactory.getConfiguration(instance).getTimeInMillis(Property.GENERAL_RPC_TIMEOUT))
-          client = ThriftUtil.getTServerClient(location, ServerConfigurationFactory.getConfiguration(instance), timeoutTracker.getTimeOut());
+        if (timeoutTracker.getTimeOut() < ServerConfigurationUtil.getConfiguration(instance).getTimeInMillis(Property.GENERAL_RPC_TIMEOUT))
+          client = ThriftUtil.getTServerClient(location, ServerConfigurationUtil.getConfiguration(instance), timeoutTracker.getTimeOut());
         else
-          client = ThriftUtil.getTServerClient(location, ServerConfigurationFactory.getConfiguration(instance));
+          client = ThriftUtil.getTServerClient(location, ServerConfigurationUtil.getConfiguration(instance));
         
         try {
           MutationSet allFailures = new MutationSet();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
index e253024..72a050a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
@@ -93,7 +93,7 @@ public class Writer {
       }
       
       try {
-        updateServer(instance, m, tabLoc.tablet_extent, tabLoc.tablet_location, credentials, ServerConfigurationFactory.getConfiguration(instance));
+        updateServer(instance, m, tabLoc.tablet_extent, tabLoc.tablet_location, credentials, ServerConfigurationUtil.getConfiguration(instance));
         return;
       } catch (NotServingTabletException e) {
         log.trace("Not serving tablet, server = " + tabLoc.tablet_location);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/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 53ac4a1..c0ef0b5 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
@@ -27,6 +27,7 @@ import java.util.Map;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.ClientSideIteratorScanner;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
@@ -43,7 +44,6 @@ import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
 import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Range;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
index ffd74a5..917b71d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
@@ -19,8 +19,8 @@ package org.apache.accumulo.core.client.mapred;
 import java.io.IOException;
 import java.util.Map.Entry;
 
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
index f6eb294..2ef9931 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
@@ -19,9 +19,9 @@ package org.apache.accumulo.core.client.mapred;
 import java.io.IOException;
 import java.util.Map;
 
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
 import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.util.format.DefaultFormatter;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
index eae6780..02512a4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
@@ -26,6 +26,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.MultiTableBatchWriter;
@@ -38,7 +39,6 @@ import org.apache.accumulo.core.client.mock.MockInstance;
 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.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.ColumnUpdate;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java
index 35ce7c7..673c5b8 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java
@@ -19,9 +19,9 @@ package org.apache.accumulo.core.client.mapred;
 import java.io.IOException;
 import java.util.Map.Entry;
 
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/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 9d8024e..5c2777d 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
@@ -31,6 +31,7 @@ import java.util.Map;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.ClientSideIteratorScanner;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
@@ -46,7 +47,6 @@ import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyExtent;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
index 0539c93..9a339be 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
@@ -19,8 +19,8 @@ package org.apache.accumulo.core.client.mapreduce;
 import java.io.IOException;
 import java.util.Map.Entry;
 
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
index e59abae..357bf38 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
@@ -22,11 +22,11 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.mapreduce.lib.util.InputConfigurator;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
index afbedca..0c924b1 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
@@ -26,6 +26,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.MultiTableBatchWriter;
@@ -38,7 +39,6 @@ import org.apache.accumulo.core.client.mock.MockInstance;
 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.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.ColumnUpdate;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
index 4734eda..37caf15 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
@@ -19,9 +19,9 @@ package org.apache.accumulo.core.client.mapreduce;
 import java.io.IOException;
 import java.util.Map.Entry;
 
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.security.Authorizations;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/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
index 9189150..0fbba98 100644
--- 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
@@ -22,12 +22,12 @@ import java.net.URISyntaxException;
 
 import org.apache.accumulo.core.Constants;
 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.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.commons.codec.binary.Base64;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/conf/ClientConfiguration.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/ClientConfiguration.java b/core/src/main/java/org/apache/accumulo/core/conf/ClientConfiguration.java
deleted file mode 100644
index 5bb95ae..0000000
--- a/core/src/main/java/org/apache/accumulo/core/conf/ClientConfiguration.java
+++ /dev/null
@@ -1,310 +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.conf;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.StringReader;
-import java.io.StringWriter;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.TreeMap;
-import java.util.UUID;
-
-import org.apache.accumulo.core.util.ArgumentChecker;
-import org.apache.commons.configuration.CompositeConfiguration;
-import org.apache.commons.configuration.Configuration;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.configuration.MapConfiguration;
-import org.apache.commons.configuration.PropertiesConfiguration;
-
-/**
- * Contains a list of property keys recognized by the Accumulo client and convenience methods for setting them.
- */
-public class ClientConfiguration extends CompositeConfiguration {
-  public static final String USER_ACCUMULO_DIR_NAME = ".accumulo";
-  public static final String USER_CONF_FILENAME = "config";
-  public static final String GLOBAL_CONF_FILENAME = "client.conf";
-
-  public enum ClientProperty {
-    RPC_SSL_TRUSTSTORE_PATH(Property.RPC_SSL_TRUSTSTORE_PATH),
-    RPC_SSL_TRUSTSTORE_PASSWORD(Property.RPC_SSL_TRUSTSTORE_PASSWORD),
-    RPC_SSL_TRUSTSTORE_TYPE(Property.RPC_SSL_TRUSTSTORE_TYPE),
-    RPC_SSL_KEYSTORE_PATH(Property.RPC_SSL_KEYSTORE_PATH),
-    RPC_SSL_KEYSTORE_PASSWORD(Property.RPC_SSL_KEYSTORE_PASSWORD),
-    RPC_SSL_KEYSTORE_TYPE(Property.RPC_SSL_KEYSTORE_TYPE),
-    RPC_USE_JSSE(Property.RPC_USE_JSSE),
-    INSTANCE_RPC_SSL_CLIENT_AUTH(Property.INSTANCE_RPC_SSL_CLIENT_AUTH),
-    INSTANCE_RPC_SSL_ENABLED(Property.INSTANCE_RPC_SSL_ENABLED),
-    INSTANCE_ZK_HOST(Property.INSTANCE_ZK_HOST),
-    INSTANCE_ZK_TIMEOUT(Property.INSTANCE_ZK_TIMEOUT),
-    INSTANCE_NAME("client.instance.name", null, PropertyType.STRING, "Name of Accumulo instance to connect to"),
-    INSTANCE_ID("client.instance.id", null, PropertyType.STRING, "UUID of Accumulo instance to connect to"),
-    ;
-
-    private String key;
-    private String defaultValue;
-    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) {
-      this.key = key;
-      this.defaultValue = defaultValue;
-      this.type = type;
-      this.description = description;
-    }
-
-    public String getKey() {
-      return key;
-    }
-
-    public String getDefaultValue() {
-      return defaultValue;
-    }
-
-    public PropertyType getType() {
-      return type;
-    }
-
-    public String getDescription() {
-      return description;
-    }
-
-    public Property getAccumuloProperty() {
-      return accumuloProperty;
-    }
-
-    public static ClientProperty getPropertyByKey(String key) {
-      for (ClientProperty prop : ClientProperty.values())
-        if (prop.getKey().equals(key))
-          return prop;
-      return null;
-    }
-  };
-
-  public ClientConfiguration(List<? extends Configuration> configs) {
-    super(configs);
-  }
-
-  public ClientConfiguration(Configuration... configs) {
-    this(Arrays.asList(configs));
-  }
-
-  public static ClientConfiguration loadDefault() {
-    return loadFromSearchPath(getDefaultSearchPath());
-  }
-
-  public static ClientConfiguration loadDefault(String overridePropertiesFilename) throws FileNotFoundException, ConfigurationException {
-    if (overridePropertiesFilename == null)
-      return loadDefault();
-    else
-      return new ClientConfiguration(new PropertiesConfiguration(overridePropertiesFilename));
-  }
-
-  private static ClientConfiguration loadFromSearchPath(List<String> paths) {
-    try {
-      List<Configuration> configs = new LinkedList<Configuration>();
-      for (String path : paths) {
-        File conf = new File(path);
-        if (conf.canRead()) {
-          configs.add(new PropertiesConfiguration(conf));
-       }
-      }
-      return new ClientConfiguration(configs);
-    } catch (ConfigurationException e) {
-      throw new IllegalStateException("Error loading client configuration", e);
-    }
-  }
-
-  public static ClientConfiguration deserialize(String serializedConfig) {
-    PropertiesConfiguration propConfig = new PropertiesConfiguration();
-    try {
-      propConfig.load(new StringReader(serializedConfig));
-    } catch (ConfigurationException e) {
-      throw new IllegalArgumentException("Error deserializing client configuration: " + serializedConfig, e);
-    }
-    return new ClientConfiguration(propConfig);
-  }
-
-  private static List<String> getDefaultSearchPath() {
-    String clientConfSearchPath = System.getenv("ACCUMULO_CLIENT_CONF_PATH");
-    List<String> clientConfPaths;
-    if (clientConfSearchPath != null) {
-      clientConfPaths = Arrays.asList(clientConfSearchPath.split(File.pathSeparator));
-    } else {
-      // if $ACCUMULO_CLIENT_CONF_PATH env isn't set, priority from top to bottom is:
-      // ~/.accumulo/config
-      // $ACCUMULO_CONF_DIR/client.conf -OR- $ACCUMULO_HOME/conf/client.conf (depending on whether $ACCUMULO_CONF_DIR is set)
-      // /etc/accumulo/client.conf
-      clientConfPaths = new LinkedList<String>();
-      clientConfPaths.add(System.getProperty("user.home") + File.separator + USER_ACCUMULO_DIR_NAME + File.separator + USER_CONF_FILENAME);
-      if (System.getenv("ACCUMULO_CONF_DIR") != null) {
-        clientConfPaths.add(System.getenv("ACCUMULO_CONF_DIR") + File.separator + GLOBAL_CONF_FILENAME);
-      } else if (System.getenv("ACCUMULO_HOME") != null) {
-        clientConfPaths.add(System.getenv("ACCUMULO_HOME") + File.separator + "conf" + File.separator + GLOBAL_CONF_FILENAME);
-      }
-      clientConfPaths.add("/etc/accumulo/" + GLOBAL_CONF_FILENAME);
-    }
-    return clientConfPaths;
-  }
-
-  public String serialize() {
-    PropertiesConfiguration propConfig = new PropertiesConfiguration();
-    propConfig.copy(this);
-    StringWriter writer = new StringWriter();
-    try {
-      propConfig.save(writer);
-    } catch (ConfigurationException e) {
-      // this should never happen
-      throw new IllegalStateException(e);
-    }
-    return writer.toString();
-  }
-
-  public String get(ClientProperty prop) {
-    if (this.containsKey(prop.getKey()))
-      return this.getString(prop.getKey());
-    else
-      return prop.getDefaultValue();
-  }
-
-  public void setProperty(ClientProperty prop, String value) {
-    this.setProperty(prop.getKey(), value);
-  }
-
-  public ClientConfiguration with(ClientProperty prop, String value) {
-    this.setProperty(prop.getKey(), value);
-    return this;
-  }
-
-  public ClientConfiguration withInstance(String instanceName) {
-    ArgumentChecker.notNull(instanceName);
-    return with(ClientProperty.INSTANCE_NAME, instanceName);
-  }
-
-  public ClientConfiguration withInstance(UUID instanceId) {
-    ArgumentChecker.notNull(instanceId);
-    return with(ClientProperty.INSTANCE_ID, instanceId.toString());
-  }
-
-  public ClientConfiguration withZkHosts(String zooKeepers) {
-    ArgumentChecker.notNull(zooKeepers);
-    return with(ClientProperty.INSTANCE_ZK_HOST, zooKeepers);
-  }
-
-  public ClientConfiguration withZkTimeout(int timeout) {
-    return with(ClientProperty.INSTANCE_ZK_TIMEOUT, String.valueOf(timeout));
-  }
-
-  public ClientConfiguration withSsl(boolean sslEnabled) {
-    return withSsl(sslEnabled, false);
-  }
-
-  public ClientConfiguration withSsl(boolean sslEnabled, boolean useJsseConfig) {
-    return with(ClientProperty.INSTANCE_RPC_SSL_ENABLED, String.valueOf(sslEnabled))
-        .with(ClientProperty.RPC_USE_JSSE, String.valueOf(useJsseConfig));
-  }
-
-  public ClientConfiguration withTruststore(String path) {
-    return withTruststore(path, null, null);
-  }
-
-  public ClientConfiguration withTruststore(String path, String password, String type) {
-    ArgumentChecker.notNull(path);
-    setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_PATH, path);
-    if (password != null)
-      setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_PASSWORD, password);
-    if (type != null)
-      setProperty(ClientProperty.RPC_SSL_TRUSTSTORE_TYPE, type);
-    return this;
-  }
-
-  public ClientConfiguration withKeystore(String path) {
-    return withKeystore(path, null, null);
-  }
-
-  public ClientConfiguration withKeystore(String path, String password, String type) {
-    ArgumentChecker.notNull(path);
-    setProperty(ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH, "true");
-    setProperty(ClientProperty.RPC_SSL_KEYSTORE_PATH, path);
-    if (password != null)
-      setProperty(ClientProperty.RPC_SSL_KEYSTORE_PASSWORD, password);
-    if (type != null)
-      setProperty(ClientProperty.RPC_SSL_KEYSTORE_TYPE, type);
-    return this;
-  }
-
-  public AccumuloConfiguration getAccumuloConfiguration() {
-    final AccumuloConfiguration defaultConf = AccumuloConfiguration.getDefaultConfiguration();
-    return new AccumuloConfiguration() {
-
-      @Override
-      public Iterator<Entry<String,String>> iterator() {
-        TreeMap<String,String> entries = new TreeMap<String,String>();
-
-        for (Entry<String,String> parentEntry : defaultConf)
-          entries.put(parentEntry.getKey(), parentEntry.getValue());
-
-        @SuppressWarnings("unchecked")
-        Iterator<String> keyIter = getKeys();
-        while (keyIter.hasNext()) {
-          String key = keyIter.next();
-          entries.put(key, getString(key));
-        }
-
-        return entries.entrySet().iterator();
-      }
-
-      @Override
-      public String get(Property property) {
-        if (containsKey(property.getKey()))
-          return getString(property.getKey());
-        else
-          return defaultConf.get(property);
-      }
-
-      @Override
-      public void getProperties(Map<String,String> props, PropertyFilter filter) {
-        for (Entry<String,String> entry : this)
-          if (filter.accept(entry.getKey()))
-            props.put(entry.getKey(), entry.getValue());
-      }
-    };
-  }
-
-  public static ClientConfiguration fromAccumuloConfiguration(AccumuloConfiguration accumuloConf) {
-    Map<String,String> props = new HashMap<String,String>();
-    for (ClientProperty prop : ClientProperty.values()) {
-      if (prop.accumuloProperty == null)
-        continue;
-      props.put(prop.getKey(), accumuloConf.get(prop.accumuloProperty));
-    }
-    return new ClientConfiguration(new MapConfiguration(props));
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
index cb86b77..25007f5 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
@@ -34,7 +34,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.AccumuloServerException;
 import org.apache.accumulo.core.client.impl.ScannerOptions;
-import org.apache.accumulo.core.client.impl.ServerConfigurationFactory;
+import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
 import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocations;
@@ -98,7 +98,7 @@ public class MetadataLocationObtainer implements TabletLocationObtainer {
       Map<String,Map<String,String>> serverSideIteratorOptions = Collections.emptyMap();
       
       boolean more = ThriftScanner.getBatchFromServer(instance, credentials, range, src.tablet_extent, src.tablet_location, encodedResults, locCols,
-          serverSideIteratorList, serverSideIteratorOptions, Constants.SCAN_BATCH_SIZE, Authorizations.EMPTY, false, ServerConfigurationFactory.getConfiguration(instance));
+          serverSideIteratorList, serverSideIteratorOptions, Constants.SCAN_BATCH_SIZE, Authorizations.EMPTY, false, ServerConfigurationUtil.getConfiguration(instance));
       
       decodeRows(encodedResults, results);
       
@@ -106,7 +106,7 @@ public class MetadataLocationObtainer implements TabletLocationObtainer {
         range = new Range(results.lastKey().followingKey(PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME), true, new Key(stopRow).followingKey(PartialKey.ROW), false);
         encodedResults.clear();
         more = ThriftScanner.getBatchFromServer(instance, credentials, range, src.tablet_extent, src.tablet_location, encodedResults, locCols,
-            serverSideIteratorList, serverSideIteratorOptions, Constants.SCAN_BATCH_SIZE, Authorizations.EMPTY, false, ServerConfigurationFactory.getConfiguration(instance));
+            serverSideIteratorList, serverSideIteratorOptions, Constants.SCAN_BATCH_SIZE, Authorizations.EMPTY, false, ServerConfigurationUtil.getConfiguration(instance));
         
         decodeRows(encodedResults, results);
       }
@@ -179,7 +179,7 @@ public class MetadataLocationObtainer implements TabletLocationObtainer {
     Map<KeyExtent,List<Range>> failures = new HashMap<KeyExtent,List<Range>>();
     try {
       TabletServerBatchReaderIterator.doLookup(instance, credentials, tserver, tabletsRanges, failures, unscanned, rr, columns, opts, Authorizations.EMPTY,
-          ServerConfigurationFactory.getConfiguration(instance));
+          ServerConfigurationUtil.getConfiguration(instance));
       if (failures.size() > 0) {
         // invalidate extents in parents cache
         if (log.isTraceEnabled())

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java b/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
index b518400..ddadae9 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java
@@ -41,19 +41,21 @@ import jline.console.history.FileHistory;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.ClientConfiguration;
+import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
 import org.apache.accumulo.core.client.mock.MockInstance;
 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.ClientConfiguration;
-import org.apache.accumulo.core.conf.SiteConfiguration;
+import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.conf.ClientConfiguration.ClientProperty;
+import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.data.thrift.TConstraintViolationSummary;
@@ -419,7 +421,8 @@ public class Shell extends ShellOptions {
       instanceName = clientConfig.get(ClientProperty.INSTANCE_NAME);
     }
     if (instanceName == null || keepers == null) {
-      AccumuloConfiguration conf = SiteConfiguration.getInstance(clientConfig.getAccumuloConfiguration());
+      AccumuloConfiguration conf = SiteConfiguration.getInstance(ServerConfigurationUtil.convertClientConfig(DefaultConfiguration.getInstance(),
+          clientConfig));
       if (instanceName == null) {
         Path instanceDir = new Path(conf.get(Property.INSTANCE_DFS_DIR), "instance_id");
         instanceId = UUID.fromString(ZooUtil.getInstanceIDFromHdfs(instanceDir));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptionsJC.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptionsJC.java b/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptionsJC.java
index 2f30a87..547da48 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptionsJC.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/ShellOptionsJC.java
@@ -24,9 +24,9 @@ import java.util.Map;
 import java.util.Scanner;
 import java.util.TreeMap;
 
+import org.apache.accumulo.core.client.ClientConfiguration;
+import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
-import org.apache.accumulo.core.conf.ClientConfiguration.ClientProperty;
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.log4j.Logger;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBaseTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBaseTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBaseTest.java
index 50fc0a9..3b5143f 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBaseTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBaseTest.java
@@ -21,14 +21,14 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 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.ClientConfiguration.ClientProperty;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
-import org.apache.accumulo.core.conf.ClientConfiguration.ClientProperty;
 import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.Level;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/test/java/org/apache/accumulo/core/conf/ClientConfigurationTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/conf/ClientConfigurationTest.java b/core/src/test/java/org/apache/accumulo/core/conf/ClientConfigurationTest.java
index 55cf9d3..40be70f 100644
--- a/core/src/test/java/org/apache/accumulo/core/conf/ClientConfigurationTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/conf/ClientConfigurationTest.java
@@ -20,7 +20,8 @@ import static org.junit.Assert.assertEquals;
 
 import java.util.Arrays;
 
-import org.apache.accumulo.core.conf.ClientConfiguration.ClientProperty;
+import org.apache.accumulo.core.client.ClientConfiguration;
+import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 import org.apache.commons.configuration.Configuration;
 import org.apache.commons.configuration.PropertiesConfiguration;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/core/src/test/java/org/apache/accumulo/core/util/shell/ShellSetInstanceTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/util/shell/ShellSetInstanceTest.java b/core/src/test/java/org/apache/accumulo/core/util/shell/ShellSetInstanceTest.java
index 5ce1320..a2769bd 100644
--- a/core/src/test/java/org/apache/accumulo/core/util/shell/ShellSetInstanceTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/util/shell/ShellSetInstanceTest.java
@@ -35,11 +35,11 @@ import java.util.UUID;
 
 import jline.console.ConsoleReader;
 
+import org.apache.accumulo.core.client.ClientConfiguration;
+import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ClientConfiguration;
-import org.apache.accumulo.core.conf.ClientConfiguration.ClientProperty;
 import org.apache.accumulo.core.conf.ConfigSanityCheck;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
@@ -154,22 +154,21 @@ public class ShellSetInstanceTest {
       expect(clientConf.get(ClientProperty.INSTANCE_NAME)).andReturn(null);
     }
 
-    AccumuloConfiguration conf = createMock(AccumuloConfiguration.class);
-    expect(clientConf.getAccumuloConfiguration()).andReturn(conf);
-
     mockStatic(ConfigSanityCheck.class);
     ConfigSanityCheck.validate(EasyMock.<AccumuloConfiguration>anyObject());
-    expectLastCall();
+    expectLastCall().atLeastOnce();
     replay(ConfigSanityCheck.class);
 
     if (!onlyHosts) {
-      expect(conf.get(Property.INSTANCE_ZK_HOST)).andReturn("host1,host2").atLeastOnce();
+      expect(clientConf.containsKey(Property.INSTANCE_ZK_HOST.getKey())).andReturn(true).atLeastOnce();
+      expect(clientConf.getString(Property.INSTANCE_ZK_HOST.getKey())).andReturn("host1,host2").atLeastOnce();
       expect(clientConf.withZkHosts("host1,host2")).andReturn(clientConf);
     }
     if (!onlyInstance) {
-      expect(conf.get(Property.INSTANCE_DFS_DIR)).andReturn("/dfs").atLeastOnce();
+      expect(clientConf.containsKey(Property.INSTANCE_DFS_DIR.getKey())).andReturn(true).atLeastOnce();
+      expect(clientConf.getString(Property.INSTANCE_DFS_DIR.getKey())).andReturn("/dfs").atLeastOnce();
     }
-    replay(conf);
+
     UUID randomUUID = null;
     if (!onlyInstance) {
       mockStatic(ZooUtil.class);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/examples/simple/src/main/java/org/apache/accumulo/examples/simple/filedata/FileDataQuery.java
----------------------------------------------------------------------
diff --git a/examples/simple/src/main/java/org/apache/accumulo/examples/simple/filedata/FileDataQuery.java b/examples/simple/src/main/java/org/apache/accumulo/examples/simple/filedata/FileDataQuery.java
index a66438e..4b12d7b 100644
--- a/examples/simple/src/main/java/org/apache/accumulo/examples/simple/filedata/FileDataQuery.java
+++ b/examples/simple/src/main/java/org/apache/accumulo/examples/simple/filedata/FileDataQuery.java
@@ -23,12 +23,12 @@ import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TokenFileWordCount.java
----------------------------------------------------------------------
diff --git a/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TokenFileWordCount.java b/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TokenFileWordCount.java
index 1114a7e..fc4b27f 100644
--- a/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TokenFileWordCount.java
+++ b/examples/simple/src/main/java/org/apache/accumulo/examples/simple/mapreduce/TokenFileWordCount.java
@@ -18,8 +18,8 @@ package org.apache.accumulo.examples.simple.mapreduce;
 
 import java.io.IOException;
 
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.util.CachedConfiguration;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/examples/simple/src/main/java/org/apache/accumulo/examples/simple/reservations/ARS.java
----------------------------------------------------------------------
diff --git a/examples/simple/src/main/java/org/apache/accumulo/examples/simple/reservations/ARS.java b/examples/simple/src/main/java/org/apache/accumulo/examples/simple/reservations/ARS.java
index 12365b6..509a674 100644
--- a/examples/simple/src/main/java/org/apache/accumulo/examples/simple/reservations/ARS.java
+++ b/examples/simple/src/main/java/org/apache/accumulo/examples/simple/reservations/ARS.java
@@ -24,13 +24,13 @@ import jline.console.ConsoleReader;
 
 import org.apache.accumulo.core.client.ConditionalWriter;
 import org.apache.accumulo.core.client.ConditionalWriter.Status;
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.ConditionalWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.data.Condition;
 import org.apache.accumulo.core.data.ConditionalMutation;
 import org.apache.accumulo.core.data.Key;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
index 0a50747..8b195ff 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
@@ -44,11 +44,11 @@ import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.ClientConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.master.thrift.MasterGoalState;
 import org.apache.accumulo.core.util.Daemon;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
index 1e1c464..43cae2d 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloInstance.java
@@ -20,9 +20,9 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.net.MalformedURLException;
 
+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.conf.ClientConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.configuration.PropertiesConfiguration;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5bd68ef9/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
----------------------------------------------------------------------
diff --git a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
index 9324da9..799c763 100644
--- a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
@@ -42,6 +42,7 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.ConditionalWriter;
 import org.apache.accumulo.core.client.ConditionalWriter.Result;
+import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.ConditionalWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
@@ -61,7 +62,6 @@ import org.apache.accumulo.core.client.mock.MockInstance;
 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.ClientConfiguration;
 import org.apache.accumulo.core.data.Column;
 import org.apache.accumulo.core.data.ConditionalMutation;
 import org.apache.accumulo.core.data.Key;


[4/6] git commit: ACCUMULO-1866 Add missing deprecation annotations

Posted by ct...@apache.org.
ACCUMULO-1866 Add missing deprecation annotations


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

Branch: refs/heads/master
Commit: de8ecd4a71cb7fda3525798c7e658c8f65efe6c2
Parents: 32b5aee
Author: Christopher Tubbs <ct...@apache.org>
Authored: Thu Nov 21 16:13:19 2013 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Thu Nov 21 16:13:19 2013 -0500

----------------------------------------------------------------------
 .../apache/accumulo/core/client/Instance.java   |  26 +-
 .../accumulo/core/client/mock/MockInstance.java |  37 +-
 .../core/client/impl/TabletLocatorImplTest.java | 755 ++++++++++---------
 .../accumulo/server/client/HdfsZooInstance.java |  67 +-
 4 files changed, 445 insertions(+), 440 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/de8ecd4a/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 ab8ca19..2cf2632 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
@@ -34,42 +34,42 @@ public interface Instance {
    * @return location in "hostname:port" form
    */
   public abstract String getRootTabletLocation();
-  
+
   /**
    * Returns the location(s) of the accumulo master and any redundant servers.
    * 
    * @return a list of locations in "hostname:port" form
    */
   public abstract List<String> getMasterLocations();
-  
+
   /**
    * Returns a unique string that identifies this instance of accumulo.
    * 
    * @return a UUID
    */
   public abstract String getInstanceID();
-  
+
   /**
    * Returns the instance name given at system initialization time.
    * 
    * @return current instance name
    */
   public abstract String getInstanceName();
-  
+
   /**
    * Returns a comma-separated list of zookeeper servers the instance is using.
    * 
    * @return the zookeeper servers this instance is using in "hostname:port" form
    */
   public abstract String getZooKeepers();
-  
+
   /**
    * Returns the zookeeper connection timeout.
    * 
    * @return the configured timeout to connect to zookeeper
    */
   public abstract int getZooKeepersSessionTimeOut();
-  
+
   /**
    * Returns a connection to accumulo.
    * 
@@ -86,7 +86,7 @@ public interface Instance {
    */
   @Deprecated
   public abstract Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Returns a connection to accumulo.
    * 
@@ -103,7 +103,7 @@ public interface Instance {
    */
   @Deprecated
   public abstract Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException;
-  
+
   /**
    * Returns a connection to this instance of accumulo.
    * 
@@ -124,18 +124,20 @@ public interface Instance {
   /**
    * Closes up the instance to free up all associated resources. You should try to reuse an Instance as much as you can because there is some location caching
    * stored which will enhance performance.
-   * @throws AccumuloException 
+   * 
+   * @throws AccumuloException
    */
   public abstract void close() throws AccumuloException;
-  
+
   /**
    * 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
    */
+  @Deprecated
   public abstract AccumuloConfiguration getConfiguration();
-  
+
   /**
    * Set the AccumuloConfiguration to use when interacting with this instance.
    * 
@@ -145,7 +147,7 @@ public interface Instance {
    */
   @Deprecated
   public abstract void setConfiguration(AccumuloConfiguration conf);
-  
+
   /**
    * Returns a connection to this instance of accumulo.
    * 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/de8ecd4a/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 5b7dcbf..837cf99 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
@@ -51,17 +51,17 @@ import org.apache.hadoop.io.Text;
  */
 
 public class MockInstance implements Instance {
-  
+
   static final String genericAddress = "localhost:1234";
   static final Map<String,MockAccumulo> instances = new HashMap<String,MockAccumulo>();
   MockAccumulo acu;
   String instanceName;
-  
+
   public MockInstance() {
     acu = new MockAccumulo(getDefaultFileSystem());
     instanceName = "mock-instance";
   }
-  
+
   static FileSystem getDefaultFileSystem() {
     try {
       Configuration conf = CachedConfiguration.getInstance();
@@ -72,11 +72,11 @@ public class MockInstance implements Instance {
       throw new RuntimeException(ex);
     }
   }
-  
+
   public MockInstance(String instanceName) {
     this(instanceName, getDefaultFileSystem());
   }
-  
+
   public MockInstance(String instanceName, FileSystem fs) {
     synchronized (instances) {
       if (instances.containsKey(instanceName))
@@ -86,70 +86,71 @@ public class MockInstance implements Instance {
     }
     this.instanceName = instanceName;
   }
-  
+
   @Override
   public String getRootTabletLocation() {
     return genericAddress;
   }
-  
+
   @Override
   public List<String> getMasterLocations() {
     return Collections.singletonList(genericAddress);
   }
-  
+
   @Override
   public String getInstanceID() {
     return "mock-instance-id";
   }
-  
+
   @Override
   public String getInstanceName() {
     return instanceName;
   }
-  
+
   @Override
   public String getZooKeepers() {
     return "localhost";
   }
-  
+
   @Override
   public int getZooKeepersSessionTimeOut() {
     return 30 * 1000;
   }
-  
+
   @Override
   @Deprecated
   public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
     return getConnector(user, new PasswordToken(pass));
   }
-  
+
   @Override
   @Deprecated
   public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
     return getConnector(user, ByteBufferUtil.toBytes(pass));
   }
-  
+
   @Override
   @Deprecated
   public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
     return getConnector(user, TextUtil.getBytes(new Text(pass.toString())));
   }
-  
+
   AccumuloConfiguration conf = null;
-  
+
+  @Deprecated
   @Override
   public AccumuloConfiguration getConfiguration() {
     if (conf == null)
       conf = AccumuloConfiguration.getDefaultConfiguration();
     return 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);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/de8ecd4a/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 7abacb8..bf92c57 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
@@ -56,103 +56,103 @@ import org.apache.accumulo.core.security.Credentials;
 import org.apache.hadoop.io.Text;
 
 public class TabletLocatorImplTest extends TestCase {
-  
+
   private static final KeyExtent RTE = RootTable.EXTENT;
   private static final KeyExtent MTE = new KeyExtent(new Text(MetadataTable.ID), null, RTE.getEndRow());
   private static Credentials credentials = null;
-  
+
   static KeyExtent nke(String t, String er, String per) {
     return new KeyExtent(new Text(t), er == null ? null : new Text(er), per == null ? null : new Text(per));
   }
-  
+
   static Range nr(String k1, boolean si, String k2, boolean ei) {
     return new Range(k1 == null ? null : new Text(k1), si, k2 == null ? null : new Text(k2), ei);
   }
-  
+
   static Range nr(String k1, String k2) {
     return new Range(k1 == null ? null : new Text(k1), k2 == null ? null : new Text(k2));
   }
-  
+
   static List<Range> nrl(Range... ranges) {
     return Arrays.asList(ranges);
   }
-  
+
   static Object[] nol(Object... objs) {
     return objs;
   }
-  
+
   @SuppressWarnings("unchecked")
   static Map<String,Map<KeyExtent,List<Range>>> createExpectedBinnings(Object... data) {
-    
+
     Map<String,Map<KeyExtent,List<Range>>> expBinnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
-    
+
     for (int i = 0; i < data.length; i += 2) {
       String loc = (String) data[i];
       Object binData[] = (Object[]) data[i + 1];
-      
+
       HashMap<KeyExtent,List<Range>> binnedKE = new HashMap<KeyExtent,List<Range>>();
-      
+
       expBinnedRanges.put(loc, binnedKE);
-      
+
       for (int j = 0; j < binData.length; j += 2) {
         KeyExtent ke = (KeyExtent) binData[j];
         List<Range> ranges = (List<Range>) binData[j + 1];
-        
+
         binnedKE.put(ke, ranges);
       }
     }
-    
+
     return expBinnedRanges;
   }
-  
+
   static TreeMap<KeyExtent,TabletLocation> createMetaCacheKE(Object... data) {
     TreeMap<KeyExtent,TabletLocation> mcke = new TreeMap<KeyExtent,TabletLocation>();
-    
+
     for (int i = 0; i < data.length; i += 2) {
       KeyExtent ke = (KeyExtent) data[i];
       String loc = (String) data[i + 1];
       mcke.put(ke, new TabletLocation(ke, loc, "1"));
     }
-    
+
     return mcke;
   }
-  
+
   static TreeMap<Text,TabletLocation> createMetaCache(Object... data) {
     TreeMap<KeyExtent,TabletLocation> mcke = createMetaCacheKE(data);
-    
+
     TreeMap<Text,TabletLocation> mc = new TreeMap<Text,TabletLocation>(TabletLocatorImpl.endRowComparator);
-    
+
     for (Entry<KeyExtent,TabletLocation> entry : mcke.entrySet()) {
       if (entry.getKey().getEndRow() == null)
         mc.put(TabletLocatorImpl.MAX_TEXT, entry.getValue());
       else
         mc.put(entry.getKey().getEndRow(), entry.getValue());
     }
-    
+
     return mc;
   }
-  
+
   static TabletLocatorImpl createLocators(TServers tservers, String rootTabLoc, String metaTabLoc, String table, TabletServerLockChecker tslc, Object... data) {
-    
+
     TreeMap<KeyExtent,TabletLocation> mcke = createMetaCacheKE(data);
-    
+
     TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
     TestInstance testInstance = new TestInstance("instance1", "tserver1");
-    
+
     RootTabletLocator rtl = new TestRootTabletLocator(testInstance);
     TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(MetadataTable.ID), rtl, ttlo, new YesLockChecker());
     TabletLocatorImpl tab1TabletCache = new TabletLocatorImpl(new Text(table), rootTabletCache, ttlo, tslc);
-    
+
     setLocation(tservers, rootTabLoc, RTE, MTE, metaTabLoc);
-    
+
     for (Entry<KeyExtent,TabletLocation> entry : mcke.entrySet()) {
       setLocation(tservers, metaTabLoc, MTE, entry.getKey(), entry.getValue().tablet_location);
     }
-    
+
     return tab1TabletCache;
-    
+
   }
-  
+
   static TabletLocatorImpl createLocators(TServers tservers, String rootTabLoc, String metaTabLoc, String table, Object... data) {
     return createLocators(tservers, rootTabLoc, metaTabLoc, table, new YesLockChecker(), data);
   }
@@ -161,376 +161,377 @@ public class TabletLocatorImplTest extends TestCase {
     TServers tservers = new TServers();
     return createLocators(tservers, "tserver1", "tserver2", table, data);
   }
-  
+
   private void runTest(Text tableName, List<Range> ranges, TabletLocatorImpl tab1TabletCache, Map<String,Map<KeyExtent,List<Range>>> expected) throws Exception {
     List<Range> failures = Collections.emptyList();
     runTest(tableName, ranges, tab1TabletCache, expected, failures);
   }
-  
+
   private void runTest(Text tableName, List<Range> ranges, TabletLocatorImpl tab1TabletCache, Map<String,Map<KeyExtent,List<Range>>> expected,
       List<Range> efailures) throws Exception {
-    
+
     Map<String,Map<KeyExtent,List<Range>>> binnedRanges = new HashMap<String,Map<KeyExtent,List<Range>>>();
     List<Range> f = tab1TabletCache.binRanges(credentials, ranges, binnedRanges);
     assertEquals(expected, binnedRanges);
-    
+
     HashSet<Range> f1 = new HashSet<Range>(f);
     HashSet<Range> f2 = new HashSet<Range>(efailures);
-    
+
     assertEquals(f2, f1);
   }
-  
+
   static Set<KeyExtent> nkes(KeyExtent... extents) {
     HashSet<KeyExtent> kes = new HashSet<KeyExtent>();
-    
+
     for (KeyExtent keyExtent : extents) {
       kes.add(keyExtent);
     }
-    
+
     return kes;
   }
-  
+
   static void runTest(TreeMap<Text,TabletLocation> mc, KeyExtent remove, Set<KeyExtent> expected) {
     // copy so same metaCache can be used for multiple test
-    
+
     mc = new TreeMap<Text,TabletLocation>(mc);
-    
+
     TabletLocatorImpl.removeOverlapping(mc, remove);
-    
+
     HashSet<KeyExtent> eic = new HashSet<KeyExtent>();
     for (TabletLocation tl : mc.values()) {
       eic.add(tl.tablet_extent);
     }
-    
+
     assertEquals(expected, eic);
   }
-  
+
   static Mutation nm(String row, String... data) {
     Mutation mut = new Mutation(new Text(row));
-    
+
     for (int i = 0; i < data.length; i++) {
       String[] cvp = data[i].split("=");
       String[] cols = cvp[0].split(":");
-      
+
       mut.put(new Text(cols[0]), new Text(cols[1]), new Value(cvp[1].getBytes()));
     }
-    
+
     return mut;
   }
-  
+
   static List<Mutation> nml(Mutation... ma) {
     return Arrays.asList(ma);
   }
-  
+
   private void runTest(TabletLocatorImpl metaCache, List<Mutation> ml, Map<String,Map<KeyExtent,List<String>>> emb, String... efailures) throws Exception {
     Map<String,TabletServerMutations<Mutation>> binnedMutations = new HashMap<String,TabletServerMutations<Mutation>>();
     List<Mutation> afailures = new ArrayList<Mutation>();
     metaCache.binMutations(credentials, ml, binnedMutations, afailures);
-    
+
     verify(emb, binnedMutations);
-    
+
     ArrayList<String> afs = new ArrayList<String>();
     ArrayList<String> efs = new ArrayList<String>(Arrays.asList(efailures));
-    
+
     for (Mutation mutation : afailures) {
       afs.add(new String(mutation.getRow()));
     }
-    
+
     Collections.sort(afs);
     Collections.sort(efs);
-    
+
     assertEquals(efs, afs);
-    
+
   }
-  
+
   private void verify(Map<String,Map<KeyExtent,List<String>>> expected, Map<String,TabletServerMutations<Mutation>> actual) {
     assertEquals(expected.keySet(), actual.keySet());
-    
+
     for (String server : actual.keySet()) {
       TabletServerMutations<Mutation> atb = actual.get(server);
       Map<KeyExtent,List<String>> etb = expected.get(server);
-      
+
       assertEquals(etb.keySet(), atb.getMutations().keySet());
-      
+
       for (KeyExtent ke : etb.keySet()) {
         ArrayList<String> eRows = new ArrayList<String>(etb.get(ke));
         ArrayList<String> aRows = new ArrayList<String>();
-        
+
         for (Mutation m : atb.getMutations().get(ke)) {
           aRows.add(new String(m.getRow()));
         }
-        
+
         Collections.sort(eRows);
         Collections.sort(aRows);
-        
+
         assertEquals(eRows, aRows);
       }
     }
-    
+
   }
-  
+
   static Map<String,Map<KeyExtent,List<String>>> cemb(Object[]... ols) {
-    
+
     Map<String,Map<KeyExtent,List<String>>> emb = new HashMap<String,Map<KeyExtent,List<String>>>();
-    
+
     for (Object[] ol : ols) {
       String row = (String) ol[0];
       String server = (String) ol[1];
       KeyExtent ke = (KeyExtent) ol[2];
-      
+
       Map<KeyExtent,List<String>> tb = emb.get(server);
       if (tb == null) {
         tb = new HashMap<KeyExtent,List<String>>();
         emb.put(server, tb);
       }
-      
+
       List<String> rl = tb.get(ke);
       if (rl == null) {
         rl = new ArrayList<String>();
         tb.put(ke, rl);
       }
-      
+
       rl.add(row);
     }
-    
+
     return emb;
   }
-  
+
   public void testRemoveOverlapping1() {
     TreeMap<Text,TabletLocation> mc = createMetaCache(nke("0", null, null), "l1");
-    
+
     runTest(mc, nke("0", "a", null), nkes());
     runTest(mc, nke("0", null, null), nkes());
     runTest(mc, nke("0", null, "a"), nkes());
-    
+
     mc = createMetaCache(nke("0", "g", null), "l1", nke("0", "r", "g"), "l1", nke("0", null, "r"), "l1");
     runTest(mc, nke("0", null, null), nkes());
-    
+
     runTest(mc, nke("0", "a", null), nkes(nke("0", "r", "g"), nke("0", null, "r")));
     runTest(mc, nke("0", "g", null), nkes(nke("0", "r", "g"), nke("0", null, "r")));
     runTest(mc, nke("0", "h", null), nkes(nke("0", null, "r")));
     runTest(mc, nke("0", "r", null), nkes(nke("0", null, "r")));
     runTest(mc, nke("0", "s", null), nkes());
-    
+
     runTest(mc, nke("0", "b", "a"), nkes(nke("0", "r", "g"), nke("0", null, "r")));
     runTest(mc, nke("0", "g", "a"), nkes(nke("0", "r", "g"), nke("0", null, "r")));
     runTest(mc, nke("0", "h", "a"), nkes(nke("0", null, "r")));
     runTest(mc, nke("0", "r", "a"), nkes(nke("0", null, "r")));
     runTest(mc, nke("0", "s", "a"), nkes());
-    
+
     runTest(mc, nke("0", "h", "g"), nkes(nke("0", "g", null), nke("0", null, "r")));
     runTest(mc, nke("0", "r", "g"), nkes(nke("0", "g", null), nke("0", null, "r")));
     runTest(mc, nke("0", "s", "g"), nkes(nke("0", "g", null)));
-    
+
     runTest(mc, nke("0", "i", "h"), nkes(nke("0", "g", null), nke("0", null, "r")));
     runTest(mc, nke("0", "r", "h"), nkes(nke("0", "g", null), nke("0", null, "r")));
     runTest(mc, nke("0", "s", "h"), nkes(nke("0", "g", null)));
-    
+
     runTest(mc, nke("0", "z", "f"), nkes());
     runTest(mc, nke("0", "z", "g"), nkes(nke("0", "g", null)));
     runTest(mc, nke("0", "z", "q"), nkes(nke("0", "g", null)));
     runTest(mc, nke("0", "z", "r"), nkes(nke("0", "g", null), nke("0", "r", "g")));
     runTest(mc, nke("0", "z", "s"), nkes(nke("0", "g", null), nke("0", "r", "g")));
-    
+
     runTest(mc, nke("0", null, "f"), nkes());
     runTest(mc, nke("0", null, "g"), nkes(nke("0", "g", null)));
     runTest(mc, nke("0", null, "q"), nkes(nke("0", "g", null)));
     runTest(mc, nke("0", null, "r"), nkes(nke("0", "g", null), nke("0", "r", "g")));
     runTest(mc, nke("0", null, "s"), nkes(nke("0", "g", null), nke("0", "r", "g")));
-    
+
   }
-  
+
   public void testRemoveOverlapping2() {
-    
+
     // test removes when cache does not contain all tablets in a table
     TreeMap<Text,TabletLocation> mc = createMetaCache(nke("0", "r", "g"), "l1", nke("0", null, "r"), "l1");
-    
+
     runTest(mc, nke("0", "a", null), nkes(nke("0", "r", "g"), nke("0", null, "r")));
     runTest(mc, nke("0", "g", null), nkes(nke("0", "r", "g"), nke("0", null, "r")));
     runTest(mc, nke("0", "h", null), nkes(nke("0", null, "r")));
     runTest(mc, nke("0", "r", null), nkes(nke("0", null, "r")));
     runTest(mc, nke("0", "s", null), nkes());
-    
+
     runTest(mc, nke("0", "b", "a"), nkes(nke("0", "r", "g"), nke("0", null, "r")));
     runTest(mc, nke("0", "g", "a"), nkes(nke("0", "r", "g"), nke("0", null, "r")));
     runTest(mc, nke("0", "h", "a"), nkes(nke("0", null, "r")));
     runTest(mc, nke("0", "r", "a"), nkes(nke("0", null, "r")));
     runTest(mc, nke("0", "s", "a"), nkes());
-    
+
     runTest(mc, nke("0", "h", "g"), nkes(nke("0", null, "r")));
-    
+
     mc = createMetaCache(nke("0", "g", null), "l1", nke("0", null, "r"), "l1");
-    
+
     runTest(mc, nke("0", "h", "g"), nkes(nke("0", "g", null), nke("0", null, "r")));
     runTest(mc, nke("0", "h", "a"), nkes(nke("0", null, "r")));
     runTest(mc, nke("0", "s", "g"), nkes(nke("0", "g", null)));
     runTest(mc, nke("0", "s", "a"), nkes());
-    
+
     mc = createMetaCache(nke("0", "g", null), "l1", nke("0", "r", "g"), "l1");
-    
+
     runTest(mc, nke("0", "z", "f"), nkes());
     runTest(mc, nke("0", "z", "g"), nkes(nke("0", "g", null)));
     runTest(mc, nke("0", "z", "q"), nkes(nke("0", "g", null)));
     runTest(mc, nke("0", "z", "r"), nkes(nke("0", "g", null), nke("0", "r", "g")));
     runTest(mc, nke("0", "z", "s"), nkes(nke("0", "g", null), nke("0", "r", "g")));
-    
+
     runTest(mc, nke("0", null, "f"), nkes());
     runTest(mc, nke("0", null, "g"), nkes(nke("0", "g", null)));
     runTest(mc, nke("0", null, "q"), nkes(nke("0", "g", null)));
     runTest(mc, nke("0", null, "r"), nkes(nke("0", "g", null), nke("0", "r", "g")));
     runTest(mc, nke("0", null, "s"), nkes(nke("0", "g", null), nke("0", "r", "g")));
   }
-  
+
   static class TestInstance implements Instance {
-    
+
     private final String iid;
     private String rtl;
-    
+
     public TestInstance(String iid, String rtl) {
       this.iid = iid;
       this.rtl = rtl;
     }
-    
+
     @Override
     public String getInstanceID() {
       return iid;
     }
-    
+
     @Override
     public String getInstanceName() {
       throw new UnsupportedOperationException();
     }
-    
+
     @Override
     public List<String> getMasterLocations() {
       throw new UnsupportedOperationException();
     }
-    
+
     @Override
     public String getRootTabletLocation() {
       return rtl;
     }
-    
+
     @Override
     public String getZooKeepers() {
       throw new UnsupportedOperationException();
     }
-    
+
     @Override
     public int getZooKeepersSessionTimeOut() {
       throw new UnsupportedOperationException();
     }
-    
+
     public void setRootTabletLocation(String rtl) {
       this.rtl = rtl;
     }
-    
+
     @Override
     @Deprecated
     public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
       throw new UnsupportedOperationException();
     }
-    
+
     @Override
     @Deprecated
     public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
       throw new UnsupportedOperationException();
     }
-    
+
     AccumuloConfiguration conf = AccumuloConfiguration.getDefaultConfiguration();
-    
+
+    @Deprecated
     @Override
     public AccumuloConfiguration getConfiguration() {
       return conf;
     }
-    
+
     @Override
     @Deprecated
     public void setConfiguration(AccumuloConfiguration conf) {
       this.conf = conf;
     }
-    
+
     @Override
     @Deprecated
     public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
       throw new UnsupportedOperationException();
     }
-    
+
     @Override
     public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
       throw new UnsupportedOperationException();
     }
-    
+
     @Override
     public void close() throws AccumuloException {
       // NOOP
     }
   }
-  
+
   static class TServers {
     private final Map<String,Map<KeyExtent,SortedMap<Key,Value>>> tservers = new HashMap<String,Map<KeyExtent,SortedMap<Key,Value>>>();
   }
-  
+
   static class TestTabletLocationObtainer implements TabletLocationObtainer {
-    
+
     private final Map<String,Map<KeyExtent,SortedMap<Key,Value>>> tservers;
-    
+
     TestTabletLocationObtainer(TServers tservers) {
       this.tservers = tservers.tservers;
     }
-    
+
     @Override
     public TabletLocations lookupTablet(Credentials credentials, TabletLocation src, Text row, Text stopRow, TabletLocator parent)
         throws AccumuloSecurityException {
-      
+
       // System.out.println("lookupTablet("+src+","+row+","+stopRow+","+ parent+")");
       // System.out.println(tservers);
 
       Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.get(src.tablet_location);
-      
+
       if (tablets == null) {
         parent.invalidateCache(src.tablet_location);
         return null;
       }
-      
+
       SortedMap<Key,Value> tabletData = tablets.get(src.tablet_extent);
-      
+
       if (tabletData == null) {
         parent.invalidateCache(src.tablet_extent);
         return null;
       }
-      
+
       // the following clip is done on a tablet, do it here to see if it throws exceptions
       src.tablet_extent.toDataRange().clip(new Range(row, true, stopRow, true));
-      
+
       Key startKey = new Key(row);
       Key stopKey = new Key(stopRow).followingKey(PartialKey.ROW);
-      
+
       SortedMap<Key,Value> results = tabletData.tailMap(startKey).headMap(stopKey);
-      
+
       return MetadataLocationObtainer.getMetadataLocationEntries(results);
     }
-    
+
     @Override
     public List<TabletLocation> lookupTablets(Credentials credentials, String tserver, Map<KeyExtent,List<Range>> map, TabletLocator parent)
         throws AccumuloSecurityException {
-      
+
       ArrayList<TabletLocation> list = new ArrayList<TabletLocation>();
-      
+
       Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.get(tserver);
-      
+
       if (tablets == null) {
         parent.invalidateCache(tserver);
         return list;
       }
-      
+
       TreeMap<Key,Value> results = new TreeMap<Key,Value>();
-      
+
       Set<Entry<KeyExtent,List<Range>>> es = map.entrySet();
       List<KeyExtent> failures = new ArrayList<KeyExtent>();
       for (Entry<KeyExtent,List<Range>> entry : es) {
         SortedMap<Key,Value> tabletData = tablets.get(entry.getKey());
-        
+
         if (tabletData == null) {
           failures.add(entry.getKey());
           continue;
@@ -542,55 +543,55 @@ public class TabletLocatorImplTest extends TestCase {
             tm = tabletData;
           else
             tm = tabletData.tailMap(range.getStartKey());
-          
+
           for (Entry<Key,Value> de : tm.entrySet()) {
             if (range.afterEndKey(de.getKey())) {
               break;
             }
-            
+
             if (range.contains(de.getKey())) {
               results.put(de.getKey(), de.getValue());
             }
           }
         }
       }
-      
+
       if (failures.size() > 0)
         parent.invalidateCache(failures);
-      
+
       return MetadataLocationObtainer.getMetadataLocationEntries(results).getLocations();
-      
+
     }
-    
+
   }
-  
+
   static class YesLockChecker implements TabletServerLockChecker {
     @Override
     public boolean isLockHeld(String tserver, String session) {
       return true;
     }
-    
+
     @Override
     public void invalidateCache(String server) {}
   }
 
   static class TestRootTabletLocator extends RootTabletLocator {
-    
+
     private Instance instance;
-    
+
     TestRootTabletLocator(Instance instance) {
       super(instance, new YesLockChecker());
       this.instance = instance;
     }
-    
+
     @Override
     protected TabletLocation getRootTabletLocation() {
       return new TabletLocation(RootTable.EXTENT, instance.getRootTabletLocation(), "1");
     }
-    
+
     @Override
     public void invalidateCache(String server) {}
-    
+
   }
 
   static void createEmptyTablet(TServers tservers, String server, KeyExtent tablet) {
@@ -599,7 +600,7 @@ public class TabletLocatorImplTest extends TestCase {
       tablets = new HashMap<KeyExtent,SortedMap<Key,Value>>();
       tservers.tservers.put(server, tablets);
     }
-    
+
     SortedMap<Key,Value> tabletData = tablets.get(tablet);
     if (tabletData == null) {
       tabletData = new TreeMap<Key,Value>();
@@ -608,22 +609,22 @@ public class TabletLocatorImplTest extends TestCase {
       throw new RuntimeException("Asked for empty tablet, but non empty tablet exists");
     }
   }
-  
+
   static void clearLocation(TServers tservers, String server, KeyExtent tablet, KeyExtent ke, String instance) {
     Map<KeyExtent,SortedMap<Key,Value>> tablets = tservers.tservers.get(server);
     if (tablets == null) {
       return;
     }
-    
+
     SortedMap<Key,Value> tabletData = tablets.get(tablet);
     if (tabletData == null) {
       return;
     }
-    
+
     Text mr = ke.getMetadataEntry();
     Key lk = new Key(mr, TabletsSection.CurrentLocationColumnFamily.NAME, new Text(instance));
     tabletData.remove(lk);
-    
+
   }
 
   static void setLocation(TServers tservers, String server, KeyExtent tablet, KeyExtent ke, String location, String instance) {
@@ -632,41 +633,41 @@ public class TabletLocatorImplTest extends TestCase {
       tablets = new HashMap<KeyExtent,SortedMap<Key,Value>>();
       tservers.tservers.put(server, tablets);
     }
-    
+
     SortedMap<Key,Value> tabletData = tablets.get(tablet);
     if (tabletData == null) {
       tabletData = new TreeMap<Key,Value>();
       tablets.put(tablet, tabletData);
     }
-    
+
     Text mr = ke.getMetadataEntry();
     Value per = KeyExtent.encodePrevEndRow(ke.getPrevEndRow());
-    
+
     if (location != null) {
       if (instance == null)
         instance = "";
       Key lk = new Key(mr, TabletsSection.CurrentLocationColumnFamily.NAME, new Text(instance));
       tabletData.put(lk, new Value(location.getBytes()));
     }
-    
+
     Key pk = new Key(mr, TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.getColumnFamily(),
         TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.getColumnQualifier());
     tabletData.put(pk, per);
   }
-  
+
   static void setLocation(TServers tservers, String server, KeyExtent tablet, KeyExtent ke, String location) {
     setLocation(tservers, server, tablet, ke, location, "");
   }
-  
+
   static void deleteServer(TServers tservers, String server) {
     tservers.tservers.remove(server);
-    
+
   }
-  
+
   private void locateTabletTest(TabletLocatorImpl cache, String row, boolean skipRow, KeyExtent expected, String server, Credentials credentials)
       throws Exception {
     TabletLocation tl = cache.locateTablet(credentials, new Text(row), skipRow, false);
-    
+
     if (expected == null) {
       if (tl != null)
         System.out.println("tl = " + tl);
@@ -677,37 +678,37 @@ public class TabletLocatorImplTest extends TestCase {
       assertEquals(expected, tl.tablet_extent);
     }
   }
-  
+
   private void locateTabletTest(TabletLocatorImpl cache, String row, KeyExtent expected, String server, Credentials credentials) throws Exception {
     locateTabletTest(cache, row, false, expected, server, credentials);
   }
-  
+
   public void test1() throws Exception {
     TServers tservers = new TServers();
     TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
     TestInstance testInstance = new TestInstance("instance1", "tserver1");
-    
+
     RootTabletLocator rtl = new TestRootTabletLocator(testInstance);
     TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(MetadataTable.ID), rtl, ttlo, new YesLockChecker());
     TabletLocatorImpl tab1TabletCache = new TabletLocatorImpl(new Text("tab1"), rootTabletCache, ttlo, new YesLockChecker());
-    
+
     locateTabletTest(tab1TabletCache, "r1", null, null, credentials);
-    
+
     KeyExtent tab1e = nke("tab1", null, null);
-    
+
     setLocation(tservers, "tserver1", RTE, MTE, "tserver2");
     setLocation(tservers, "tserver2", MTE, tab1e, "tserver3");
-    
+
     locateTabletTest(tab1TabletCache, "r1", tab1e, "tserver3", credentials);
     locateTabletTest(tab1TabletCache, "r2", tab1e, "tserver3", credentials);
-    
+
     // simulate a split
     KeyExtent tab1e1 = nke("tab1", "g", null);
     KeyExtent tab1e2 = nke("tab1", null, "g");
-    
+
     setLocation(tservers, "tserver2", MTE, tab1e1, "tserver4");
     setLocation(tservers, "tserver2", MTE, tab1e2, "tserver5");
-    
+
     locateTabletTest(tab1TabletCache, "r1", tab1e, "tserver3", credentials);
     tab1TabletCache.invalidateCache(tab1e);
     locateTabletTest(tab1TabletCache, "r1", tab1e2, "tserver5", credentials);
@@ -715,7 +716,7 @@ public class TabletLocatorImplTest extends TestCase {
     locateTabletTest(tab1TabletCache, "a", true, tab1e1, "tserver4", credentials);
     locateTabletTest(tab1TabletCache, "g", tab1e1, "tserver4", credentials);
     locateTabletTest(tab1TabletCache, "g", true, tab1e2, "tserver5", credentials);
-    
+
     // simulate a partial split
     KeyExtent tab1e22 = nke("tab1", null, "m");
     setLocation(tservers, "tserver2", MTE, tab1e22, "tserver6");
@@ -729,72 +730,72 @@ public class TabletLocatorImplTest extends TestCase {
     locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credentials);
     locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver7", credentials);
     locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credentials);
-    
+
     // simulate a migration
     setLocation(tservers, "tserver2", MTE, tab1e21, "tserver8");
     tab1TabletCache.invalidateCache(tab1e21);
     locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credentials);
     locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credentials);
     locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credentials);
-    
+
     // simulate a server failure
     setLocation(tservers, "tserver2", MTE, tab1e21, "tserver9");
     tab1TabletCache.invalidateCache("tserver8");
     locateTabletTest(tab1TabletCache, "r1", tab1e22, "tserver6", credentials);
     locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver9", credentials);
     locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver4", credentials);
-    
+
     // simulate all servers failing
     deleteServer(tservers, "tserver1");
     deleteServer(tservers, "tserver2");
     tab1TabletCache.invalidateCache("tserver4");
     tab1TabletCache.invalidateCache("tserver6");
     tab1TabletCache.invalidateCache("tserver9");
-    
+
     locateTabletTest(tab1TabletCache, "r1", null, null, credentials);
     locateTabletTest(tab1TabletCache, "h", null, null, credentials);
     locateTabletTest(tab1TabletCache, "a", null, null, credentials);
-    
+
     testInstance.setRootTabletLocation("tserver4");
     setLocation(tservers, "tserver4", RTE, MTE, "tserver5");
     setLocation(tservers, "tserver5", MTE, tab1e1, "tserver1");
     setLocation(tservers, "tserver5", MTE, tab1e21, "tserver2");
     setLocation(tservers, "tserver5", MTE, tab1e22, "tserver3");
-    
+
     locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver1", credentials);
     locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver2", credentials);
     locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver3", credentials);
-    
+
     // simulate the !METADATA table splitting
     KeyExtent mte1 = new KeyExtent(new Text(MetadataTable.ID), tab1e21.getMetadataEntry(), RTE.getEndRow());
     KeyExtent mte2 = new KeyExtent(new Text(MetadataTable.ID), null, tab1e21.getMetadataEntry());
-    
+
     setLocation(tservers, "tserver4", RTE, mte1, "tserver5");
     setLocation(tservers, "tserver4", RTE, mte2, "tserver6");
     deleteServer(tservers, "tserver5");
     setLocation(tservers, "tserver5", mte1, tab1e1, "tserver7");
     setLocation(tservers, "tserver5", mte1, tab1e21, "tserver8");
     setLocation(tservers, "tserver6", mte2, tab1e22, "tserver9");
-    
+
     tab1TabletCache.invalidateCache(tab1e1);
     tab1TabletCache.invalidateCache(tab1e21);
     tab1TabletCache.invalidateCache(tab1e22);
-    
+
     locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver7", credentials);
     locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credentials);
     locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credentials);
-    
+
     // simulate metadata and regular server down and the reassigned
     deleteServer(tservers, "tserver5");
     tab1TabletCache.invalidateCache("tserver7");
     locateTabletTest(tab1TabletCache, "a", null, null, credentials);
     locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credentials);
     locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credentials);
-    
+
     setLocation(tservers, "tserver4", RTE, mte1, "tserver10");
     setLocation(tservers, "tserver10", mte1, tab1e1, "tserver7");
     setLocation(tservers, "tserver10", mte1, tab1e21, "tserver8");
-    
+
     locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver7", credentials);
     locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credentials);
     locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credentials);
@@ -803,465 +804,465 @@ public class TabletLocatorImplTest extends TestCase {
     locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver2", credentials);
     locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver8", credentials);
     locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credentials);
-    
+
     // simulate a hole in the !METADATA table, caused by a partial split
     KeyExtent mte11 = new KeyExtent(new Text(MetadataTable.ID), tab1e1.getMetadataEntry(), RTE.getEndRow());
     KeyExtent mte12 = new KeyExtent(new Text(MetadataTable.ID), tab1e21.getMetadataEntry(), tab1e1.getMetadataEntry());
     deleteServer(tservers, "tserver10");
     setLocation(tservers, "tserver4", RTE, mte12, "tserver10");
     setLocation(tservers, "tserver10", mte12, tab1e21, "tserver12");
-    
+
     // at this point should be no info in !METADATA about tab1e1
     tab1TabletCache.invalidateCache(tab1e1);
     tab1TabletCache.invalidateCache(tab1e21);
     locateTabletTest(tab1TabletCache, "a", null, null, credentials);
     locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver12", credentials);
     locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credentials);
-    
+
     setLocation(tservers, "tserver4", RTE, mte11, "tserver5");
     setLocation(tservers, "tserver5", mte11, tab1e1, "tserver13");
-    
+
     locateTabletTest(tab1TabletCache, "a", tab1e1, "tserver13", credentials);
     locateTabletTest(tab1TabletCache, "h", tab1e21, "tserver12", credentials);
     locateTabletTest(tab1TabletCache, "r", tab1e22, "tserver9", credentials);
   }
-  
+
   public void test2() throws Exception {
     TServers tservers = new TServers();
     TabletLocatorImpl metaCache = createLocators(tservers, "tserver1", "tserver2", "foo");
-    
+
     KeyExtent ke1 = nke("foo", "m", null);
     KeyExtent ke2 = nke("foo", null, "m");
-    
+
     setLocation(tservers, "tserver2", MTE, ke1, null);
     setLocation(tservers, "tserver2", MTE, ke2, "L1");
-    
+
     locateTabletTest(metaCache, "a", null, null, credentials);
     locateTabletTest(metaCache, "r", ke2, "L1", credentials);
-    
+
     setLocation(tservers, "tserver2", MTE, ke1, "L2");
-    
+
     locateTabletTest(metaCache, "a", ke1, "L2", credentials);
     locateTabletTest(metaCache, "r", ke2, "L1", credentials);
   }
-  
+
   public void testBinRanges1() throws Exception {
     Text tableName = new Text("foo");
-    
+
     TabletLocatorImpl metaCache = createLocators("foo", nke("foo", null, null), "l1");
-    
+
     List<Range> ranges = nrl(nr(null, null));
     Map<String,Map<KeyExtent,List<Range>>> expected = createExpectedBinnings("l1", nol(nke("foo", null, null), nrl(nr(null, null)))
-    
+
     );
-    
+
     runTest(tableName, ranges, metaCache, expected);
-    
+
     ranges = nrl(nr("a", null));
     expected = createExpectedBinnings("l1", nol(nke("foo", null, null), nrl(nr("a", null)))
-    
+
     );
-    
+
     runTest(tableName, ranges, metaCache, expected);
-    
+
     ranges = nrl(nr(null, "b"));
     expected = createExpectedBinnings("l1", nol(nke("foo", null, null), nrl(nr(null, "b")))
-    
+
     );
-    
+
     runTest(tableName, ranges, metaCache, expected);
   }
-  
+
   public void testBinRanges2() throws Exception {
-    
+
     Text tableName = new Text("foo");
-    
+
     List<Range> ranges = nrl(nr(null, null));
     TabletLocatorImpl metaCache = createLocators("foo", nke("foo", "g", null), "l1", nke("foo", null, "g"), "l2");
-    
+
     Map<String,Map<KeyExtent,List<Range>>> expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr(null, null))), "l2",
         nol(nke("foo", null, "g"), nrl(nr(null, null)))
-    
+
     );
-    
+
     runTest(tableName, ranges, metaCache, expected);
   }
-  
+
   public void testBinRanges3() throws Exception {
-    
+
     Text tableName = new Text("foo");
-    
+
     // test with three tablets and a range that covers the whole table
     List<Range> ranges = nrl(nr(null, null));
     TabletLocatorImpl metaCache = createLocators("foo", nke("foo", "g", null), "l1", nke("foo", "m", "g"), "l2", nke("foo", null, "m"), "l2");
-    
+
     Map<String,Map<KeyExtent,List<Range>>> expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr(null, null))), "l2",
         nol(nke("foo", "m", "g"), nrl(nr(null, null)), nke("foo", null, "m"), nrl(nr(null, null)))
-    
+
     );
-    
+
     runTest(tableName, ranges, metaCache, expected);
-    
+
     // test with three tablets where one range falls within the first tablet and last two ranges fall within the last tablet
     ranges = nrl(nr(null, "c"), nr("s", "y"), nr("z", null));
     expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr(null, "c"))), "l2", nol(nke("foo", null, "m"), nrl(nr("s", "y"), nr("z", null)))
-    
+
     );
-    
+
     runTest(tableName, ranges, metaCache, expected);
-    
+
     // test is same as above, but has an additional range that spans the first two tablets
     ranges = nrl(nr(null, "c"), nr("f", "i"), nr("s", "y"), nr("z", null));
     expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr(null, "c"), nr("f", "i"))), "l2",
         nol(nke("foo", "m", "g"), nrl(nr("f", "i")), nke("foo", null, "m"), nrl(nr("s", "y"), nr("z", null)))
-    
+
     );
-    
+
     runTest(tableName, ranges, metaCache, expected);
-    
+
     // test where start of range is not inclusive and same as tablet endrow
     ranges = nrl(nr("g", false, "m", true));
     expected = createExpectedBinnings("l2", nol(nke("foo", "m", "g"), nrl(nr("g", false, "m", true)))
-    
+
     );
-    
+
     runTest(tableName, ranges, metaCache, expected);
-    
+
     // test where start of range is inclusive and same as tablet endrow
     ranges = nrl(nr("g", true, "m", true));
     expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr("g", true, "m", true))), "l2",
         nol(nke("foo", "m", "g"), nrl(nr("g", true, "m", true)))
-    
+
     );
-    
+
     runTest(tableName, ranges, metaCache, expected);
-    
+
     ranges = nrl(nr("g", true, "m", false));
     expected = createExpectedBinnings("l1", nol(nke("foo", "g", null), nrl(nr("g", true, "m", false))), "l2",
         nol(nke("foo", "m", "g"), nrl(nr("g", true, "m", false)))
-    
+
     );
-    
+
     runTest(tableName, ranges, metaCache, expected);
-    
+
     ranges = nrl(nr("g", false, "m", false));
     expected = createExpectedBinnings("l2", nol(nke("foo", "m", "g"), nrl(nr("g", false, "m", false)))
-    
+
     );
-    
+
     runTest(tableName, ranges, metaCache, expected);
   }
-  
+
   public void testBinRanges4() throws Exception {
     Text tableName = new Text("foo");
-    
+
     List<Range> ranges = nrl(new Range(new Text("1")));
     TabletLocatorImpl metaCache = createLocators("foo", nke("foo", "0", null), "l1", nke("foo", "1", "0"), "l2", nke("foo", "2", "1"), "l3",
         nke("foo", "3", "2"), "l4", nke("foo", null, "3"), "l5");
-    
+
     Map<String,Map<KeyExtent,List<Range>>> expected = createExpectedBinnings("l2", nol(nke("foo", "1", "0"), nrl(new Range(new Text("1"))))
-    
+
     );
-    
+
     runTest(tableName, ranges, metaCache, expected);
-    
+
     Key rowColKey = new Key(new Text("3"), new Text("cf1"), new Text("cq1"));
     Range range = new Range(rowColKey, true, new Key(new Text("3")).followingKey(PartialKey.ROW), false);
-    
+
     ranges = nrl(range);
     Map<String,Map<KeyExtent,List<Range>>> expected4 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(range))
-    
+
     );
-    
+
     runTest(tableName, ranges, metaCache, expected4, nrl());
-    
+
     range = new Range(rowColKey, true, new Key(new Text("3")).followingKey(PartialKey.ROW), true);
-    
+
     ranges = nrl(range);
     Map<String,Map<KeyExtent,List<Range>>> expected5 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(range)), "l5",
         nol(nke("foo", null, "3"), nrl(range))
-    
+
     );
-    
+
     runTest(tableName, ranges, metaCache, expected5, nrl());
-    
+
     range = new Range(new Text("2"), false, new Text("3"), false);
     ranges = nrl(range);
     Map<String,Map<KeyExtent,List<Range>>> expected6 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(range))
-    
+
     );
     runTest(tableName, ranges, metaCache, expected6, nrl());
-    
+
     range = new Range(new Text("2"), true, new Text("3"), false);
     ranges = nrl(range);
     Map<String,Map<KeyExtent,List<Range>>> expected7 = createExpectedBinnings("l3", nol(nke("foo", "2", "1"), nrl(range)), "l4",
         nol(nke("foo", "3", "2"), nrl(range))
-    
+
     );
     runTest(tableName, ranges, metaCache, expected7, nrl());
-    
+
     range = new Range(new Text("2"), false, new Text("3"), true);
     ranges = nrl(range);
     Map<String,Map<KeyExtent,List<Range>>> expected8 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(range))
-    
+
     );
     runTest(tableName, ranges, metaCache, expected8, nrl());
-    
+
     range = new Range(new Text("2"), true, new Text("3"), true);
     ranges = nrl(range);
     Map<String,Map<KeyExtent,List<Range>>> expected9 = createExpectedBinnings("l3", nol(nke("foo", "2", "1"), nrl(range)), "l4",
         nol(nke("foo", "3", "2"), nrl(range))
-    
+
     );
     runTest(tableName, ranges, metaCache, expected9, nrl());
-    
+
   }
-  
+
   public void testBinRanges5() throws Exception {
     // Test binning when there is a hole in the !METADATA information
     Text tableName = new Text("foo");
-    
+
     List<Range> ranges = nrl(new Range(new Text("1")));
     TabletLocatorImpl metaCache = createLocators("foo", nke("foo", "0", null), "l1", nke("foo", "1", "0"), "l2", nke("foo", "3", "2"), "l4",
         nke("foo", null, "3"), "l5");
-    
+
     Map<String,Map<KeyExtent,List<Range>>> expected1 = createExpectedBinnings("l2", nol(nke("foo", "1", "0"), nrl(new Range(new Text("1"))))
-    
+
     );
-    
+
     runTest(tableName, ranges, metaCache, expected1);
-    
+
     ranges = nrl(new Range(new Text("2")), new Range(new Text("11")));
     Map<String,Map<KeyExtent,List<Range>>> expected2 = createExpectedBinnings();
-    
+
     runTest(tableName, ranges, metaCache, expected2, ranges);
-    
+
     ranges = nrl(new Range(new Text("1")), new Range(new Text("2")));
-    
+
     runTest(tableName, ranges, metaCache, expected1, nrl(new Range(new Text("2"))));
-    
+
     ranges = nrl(nr("0", "2"), nr("3", "4"));
     Map<String,Map<KeyExtent,List<Range>>> expected3 = createExpectedBinnings("l4", nol(nke("foo", "3", "2"), nrl(nr("3", "4"))), "l5",
         nol(nke("foo", null, "3"), nrl(nr("3", "4")))
-    
+
     );
-    
+
     runTest(tableName, ranges, metaCache, expected3, nrl(nr("0", "2")));
-    
+
     ranges = nrl(nr("0", "1"), nr("0", "11"), nr("1", "2"), nr("0", "4"), nr("2", "4"), nr("21", "4"));
     Map<String,Map<KeyExtent,List<Range>>> expected4 = createExpectedBinnings("l1", nol(nke("foo", "0", null), nrl(nr("0", "1"))), "l2",
         nol(nke("foo", "1", "0"), nrl(nr("0", "1"))), "l4", nol(nke("foo", "3", "2"), nrl(nr("21", "4"))), "l5", nol(nke("foo", null, "3"), nrl(nr("21", "4")))
-    
+
     );
-    
+
     runTest(tableName, ranges, metaCache, expected4, nrl(nr("0", "11"), nr("1", "2"), nr("0", "4"), nr("2", "4")));
   }
-  
+
   public void testBinMutations1() throws Exception {
     // one tablet table
     KeyExtent ke1 = nke("foo", null, null);
     TabletLocatorImpl metaCache = createLocators("foo", ke1, "l1");
-    
+
     List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("c", "cf1:cq1=v3", "cf1:cq2=v4"));
     Map<String,Map<KeyExtent,List<String>>> emb = cemb(nol("a", "l1", ke1), nol("c", "l1", ke1));
     runTest(metaCache, ml, emb);
-    
+
     ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"));
     emb = cemb(nol("a", "l1", ke1));
     runTest(metaCache, ml, emb);
-    
+
     ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("a", "cf1:cq3=v3"));
     emb = cemb(nol("a", "l1", ke1), nol("a", "l1", ke1));
     runTest(metaCache, ml, emb);
-    
+
   }
-  
+
   public void testBinMutations2() throws Exception {
     // no tablets for table
     TabletLocatorImpl metaCache = createLocators("foo");
-    
+
     List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("c", "cf1:cq1=v3", "cf1:cq2=v4"));
     Map<String,Map<KeyExtent,List<String>>> emb = cemb();
     runTest(metaCache, ml, emb, "a", "c");
   }
-  
+
   public void testBinMutations3() throws Exception {
     // three tablet table
     KeyExtent ke1 = nke("foo", "h", null);
     KeyExtent ke2 = nke("foo", "t", "h");
     KeyExtent ke3 = nke("foo", null, "t");
-    
+
     TabletLocatorImpl metaCache = createLocators("foo", ke1, "l1", ke2, "l2", ke3, "l3");
-    
+
     List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("i", "cf1:cq1=v3", "cf1:cq2=v4"));
     Map<String,Map<KeyExtent,List<String>>> emb = cemb(nol("a", "l1", ke1), nol("i", "l2", ke2));
     runTest(metaCache, ml, emb);
-    
+
     ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"));
     emb = cemb(nol("a", "l1", ke1));
     runTest(metaCache, ml, emb);
-    
+
     ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("a", "cf1:cq3=v3"));
     emb = cemb(nol("a", "l1", ke1), nol("a", "l1", ke1));
     runTest(metaCache, ml, emb);
-    
+
     ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"));
     emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3));
     runTest(metaCache, ml, emb);
-    
+
     ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"), nm("z", "cf1:cq4=v4"));
     emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3), nol("z", "l3", ke3));
     runTest(metaCache, ml, emb);
-    
+
     ml = nml(nm("h", "cf1:cq1=v1", "cf1:cq2=v2"), nm("t", "cf1:cq1=v1", "cf1:cq2=v2"));
     emb = cemb(nol("h", "l1", ke1), nol("t", "l2", ke2));
     runTest(metaCache, ml, emb);
   }
-  
+
   public void testBinMutations4() throws Exception {
     // three table with hole
     KeyExtent ke1 = nke("foo", "h", null);
-    
+
     KeyExtent ke3 = nke("foo", null, "t");
-    
+
     TabletLocatorImpl metaCache = createLocators("foo", ke1, "l1", ke3, "l3");
-    
+
     List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("i", "cf1:cq1=v3", "cf1:cq2=v4"));
     Map<String,Map<KeyExtent,List<String>>> emb = cemb(nol("a", "l1", ke1));
     runTest(metaCache, ml, emb, "i");
-    
+
     ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"));
     emb = cemb(nol("a", "l1", ke1));
     runTest(metaCache, ml, emb);
-    
+
     ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("a", "cf1:cq3=v3"));
     emb = cemb(nol("a", "l1", ke1), nol("a", "l1", ke1));
     runTest(metaCache, ml, emb);
-    
+
     ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"));
     emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3));
     runTest(metaCache, ml, emb);
-    
+
     ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"), nm("z", "cf1:cq4=v4"));
     emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3), nol("z", "l3", ke3));
     runTest(metaCache, ml, emb);
-    
+
     ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"), nm("z", "cf1:cq4=v4"), nm("t", "cf1:cq5=v5"));
     emb = cemb(nol("a", "l1", ke1), nol("w", "l3", ke3), nol("z", "l3", ke3));
     runTest(metaCache, ml, emb, "t");
   }
-  
+
   public void testBinSplit() throws Exception {
     // try binning mutations and ranges when a tablet splits
-    
+
     for (int i = 0; i < 3; i++) {
       // when i == 0 only test binning mutations
       // when i == 1 only test binning ranges
       // when i == 2 test both
-      
+
       KeyExtent ke1 = nke("foo", null, null);
       TServers tservers = new TServers();
       TabletLocatorImpl metaCache = createLocators(tservers, "tserver1", "tserver2", "foo", ke1, "l1");
-      
+
       List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("m", "cf1:cq1=v3", "cf1:cq2=v4"), nm("z", "cf1:cq1=v5"));
       Map<String,Map<KeyExtent,List<String>>> emb = cemb(nol("a", "l1", ke1), nol("m", "l1", ke1), nol("z", "l1", ke1));
       if (i == 0 || i == 2)
         runTest(metaCache, ml, emb);
-      
+
       List<Range> ranges = nrl(new Range(new Text("a")), new Range(new Text("m")), new Range(new Text("z")));
-      
+
       Map<String,Map<KeyExtent,List<Range>>> expected1 = createExpectedBinnings("l1", nol(nke("foo", null, null), ranges)
-      
+
       );
-      
+
       if (i == 1 || i == 2)
         runTest(new Text("foo"), ranges, metaCache, expected1);
-      
+
       KeyExtent ke11 = nke("foo", "n", null);
       KeyExtent ke12 = nke("foo", null, "n");
-      
+
       setLocation(tservers, "tserver2", MTE, ke12, "l2");
-      
+
       metaCache.invalidateCache(ke1);
-      
+
       emb = cemb(nol("z", "l2", ke12));
       if (i == 0 || i == 2)
         runTest(metaCache, ml, emb, "a", "m");
-      
+
       Map<String,Map<KeyExtent,List<Range>>> expected2 = createExpectedBinnings("l2", nol(nke("foo", null, "n"), nrl(new Range(new Text("z"))))
-      
+
       );
-      
+
       if (i == 1 || i == 2)
         runTest(new Text("foo"), ranges, metaCache, expected2, nrl(new Range(new Text("a")), new Range(new Text("m"))));
-      
+
       setLocation(tservers, "tserver2", MTE, ke11, "l3");
       emb = cemb(nol("a", "l3", ke11), nol("m", "l3", ke11), nol("z", "l2", ke12));
       if (i == 0 || i == 2)
         runTest(metaCache, ml, emb);
-      
+
       Map<String,Map<KeyExtent,List<Range>>> expected3 = createExpectedBinnings("l2", nol(nke("foo", null, "n"), nrl(new Range(new Text("z")))), "l3",
           nol(nke("foo", "n", null), nrl(new Range(new Text("a")), new Range(new Text("m"))))
-      
+
       );
-      
+
       if (i == 1 || i == 2)
         runTest(new Text("foo"), ranges, metaCache, expected3);
     }
   }
-  
+
   public void testBug1() throws Exception {
     // a bug that occurred while running continuous ingest
     KeyExtent mte1 = new KeyExtent(new Text(MetadataTable.ID), new Text("0;0bc"), RTE.getEndRow());
     KeyExtent mte2 = new KeyExtent(new Text(MetadataTable.ID), null, new Text("0;0bc"));
-    
+
     TServers tservers = new TServers();
     TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
     TestInstance testInstance = new TestInstance("instance1", "tserver1");
-    
+
     RootTabletLocator rtl = new TestRootTabletLocator(testInstance);
     TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(MetadataTable.ID), rtl, ttlo, new YesLockChecker());
     TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(new Text("0"), rootTabletCache, ttlo, new YesLockChecker());
-    
+
     setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
     setLocation(tservers, "tserver1", RTE, mte2, "tserver3");
-    
+
     // create two tablets that straddle a !METADATA split point
     KeyExtent ke1 = new KeyExtent(new Text("0"), new Text("0bbf20e"), null);
     KeyExtent ke2 = new KeyExtent(new Text("0"), new Text("0bc0756"), new Text("0bbf20e"));
-    
+
     setLocation(tservers, "tserver2", mte1, ke1, "tserver4");
     setLocation(tservers, "tserver3", mte2, ke2, "tserver5");
-    
+
     // look up something that comes after the last entry in mte1
     locateTabletTest(tab0TabletCache, "0bbff", ke2, "tserver5", credentials);
   }
-  
+
   public void testBug2() throws Exception {
     // a bug that occurred while running a functional test
     KeyExtent mte1 = new KeyExtent(new Text(MetadataTable.ID), new Text("~"), RTE.getEndRow());
     KeyExtent mte2 = new KeyExtent(new Text(MetadataTable.ID), null, new Text("~"));
-    
+
     TServers tservers = new TServers();
     TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
     TestInstance testInstance = new TestInstance("instance1", "tserver1");
-    
+
     RootTabletLocator rtl = new TestRootTabletLocator(testInstance);
     TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(MetadataTable.ID), rtl, ttlo, new YesLockChecker());
     TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(new Text("0"), rootTabletCache, ttlo, new YesLockChecker());
-    
+
     setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
     setLocation(tservers, "tserver1", RTE, mte2, "tserver3");
-    
+
     // create the ~ tablet so it exists
     Map<KeyExtent,SortedMap<Key,Value>> ts3 = new HashMap<KeyExtent,SortedMap<Key,Value>>();
     ts3.put(mte2, new TreeMap<Key,Value>());
     tservers.tservers.put("tserver3", ts3);
-    
+
     assertNull(tab0TabletCache.locateTablet(credentials, new Text("row_0000000000"), false, false));
-    
+
   }
-  
+
   // this test reproduces a problem where empty metadata tablets, that were created by user tablets being merged away, caused locating tablets to fail
   public void testBug3() throws Exception {
     KeyExtent mte1 = new KeyExtent(new Text(MetadataTable.ID), new Text("1;c"), RTE.getEndRow());
@@ -1269,172 +1270,172 @@ public class TabletLocatorImplTest extends TestCase {
     KeyExtent mte3 = new KeyExtent(new Text(MetadataTable.ID), new Text("1;j"), new Text("1;f"));
     KeyExtent mte4 = new KeyExtent(new Text(MetadataTable.ID), new Text("1;r"), new Text("1;j"));
     KeyExtent mte5 = new KeyExtent(new Text(MetadataTable.ID), null, new Text("1;r"));
-    
+
     KeyExtent ke1 = new KeyExtent(new Text("1"), null, null);
-    
+
     TServers tservers = new TServers();
     TestTabletLocationObtainer ttlo = new TestTabletLocationObtainer(tservers);
     TestInstance testInstance = new TestInstance("instance1", "tserver1");
-    
+
     RootTabletLocator rtl = new TestRootTabletLocator(testInstance);
-    
+
     TabletLocatorImpl rootTabletCache = new TabletLocatorImpl(new Text(MetadataTable.ID), rtl, ttlo, new YesLockChecker());
     TabletLocatorImpl tab0TabletCache = new TabletLocatorImpl(new Text("1"), rootTabletCache, ttlo, new YesLockChecker());
-    
+
     setLocation(tservers, "tserver1", RTE, mte1, "tserver2");
     setLocation(tservers, "tserver1", RTE, mte2, "tserver3");
     setLocation(tservers, "tserver1", RTE, mte3, "tserver4");
     setLocation(tservers, "tserver1", RTE, mte4, "tserver5");
     setLocation(tservers, "tserver1", RTE, mte5, "tserver6");
-    
+
     createEmptyTablet(tservers, "tserver2", mte1);
     createEmptyTablet(tservers, "tserver3", mte2);
     createEmptyTablet(tservers, "tserver4", mte3);
     createEmptyTablet(tservers, "tserver5", mte4);
     setLocation(tservers, "tserver6", mte5, ke1, "tserver7");
-    
+
     locateTabletTest(tab0TabletCache, "a", ke1, "tserver7", credentials);
-    
+
   }
-  
+
   public void testAccumulo1248() throws Exception {
     TServers tservers = new TServers();
     TabletLocatorImpl metaCache = createLocators(tservers, "tserver1", "tserver2", "foo");
-    
+
     KeyExtent ke1 = nke("foo", null, null);
-    
+
     // set two locations for a tablet, this is not supposed to happen. The metadata cache should throw an exception if it sees this rather than caching one of
     // the locations.
     setLocation(tservers, "tserver2", MTE, ke1, "L1", "I1");
     setLocation(tservers, "tserver2", MTE, ke1, "L2", "I2");
-    
+
     try {
       metaCache.locateTablet(credentials, new Text("a"), false, false);
       assertTrue(false);
     } catch (Exception e) {
-      
+
     }
-    
+
   }
-  
+
   public void testLostLock() throws Exception {
-    
+
     final HashSet<String> activeLocks = new HashSet<String>();
-    
+
     TServers tservers = new TServers();
     TabletLocatorImpl metaCache = createLocators(tservers, "tserver1", "tserver2", "foo", new TabletServerLockChecker() {
       @Override
       public boolean isLockHeld(String tserver, String session) {
         return activeLocks.contains(tserver + ":" + session);
       }
-      
+
       @Override
       public void invalidateCache(String server) {}
     });
-    
+
     KeyExtent ke1 = nke("foo", null, null);
     setLocation(tservers, "tserver2", MTE, ke1, "L1", "5");
-    
+
     activeLocks.add("L1:5");
-    
+
     locateTabletTest(metaCache, "a", ke1, "L1", credentials);
     locateTabletTest(metaCache, "a", ke1, "L1", credentials);
 
     activeLocks.clear();
-    
+
     locateTabletTest(metaCache, "a", null, null, credentials);
     locateTabletTest(metaCache, "a", null, null, credentials);
     locateTabletTest(metaCache, "a", null, null, credentials);
-    
+
     clearLocation(tservers, "tserver2", MTE, ke1, "5");
     setLocation(tservers, "tserver2", MTE, ke1, "L2", "6");
-    
+
     activeLocks.add("L2:6");
-    
+
     locateTabletTest(metaCache, "a", ke1, "L2", credentials);
     locateTabletTest(metaCache, "a", ke1, "L2", credentials);
 
     clearLocation(tservers, "tserver2", MTE, ke1, "6");
-    
+
     locateTabletTest(metaCache, "a", ke1, "L2", credentials);
-    
+
     setLocation(tservers, "tserver2", MTE, ke1, "L3", "7");
 
     locateTabletTest(metaCache, "a", ke1, "L2", credentials);
-    
+
     activeLocks.clear();
-    
+
     locateTabletTest(metaCache, "a", null, null, credentials);
     locateTabletTest(metaCache, "a", null, null, credentials);
 
     activeLocks.add("L3:7");
-    
+
     locateTabletTest(metaCache, "a", ke1, "L3", credentials);
     locateTabletTest(metaCache, "a", ke1, "L3", credentials);
-    
+
     List<Mutation> ml = nml(nm("a", "cf1:cq1=v1", "cf1:cq2=v2"), nm("w", "cf1:cq3=v3"));
     Map<String,Map<KeyExtent,List<String>>> emb = cemb(nol("a", "L3", ke1), nol("w", "L3", ke1));
     runTest(metaCache, ml, emb);
 
     clearLocation(tservers, "tserver2", MTE, ke1, "7");
-    
+
     runTest(metaCache, ml, emb);
-    
+
     activeLocks.clear();
-    
+
     emb.clear();
-    
+
     runTest(metaCache, ml, emb, "a", "w");
     runTest(metaCache, ml, emb, "a", "w");
-    
+
     KeyExtent ke11 = nke("foo", "m", null);
     KeyExtent ke12 = nke("foo", null, "m");
-    
+
     setLocation(tservers, "tserver2", MTE, ke11, "L1", "8");
     setLocation(tservers, "tserver2", MTE, ke12, "L2", "9");
-    
+
     runTest(metaCache, ml, emb, "a", "w");
-    
+
     activeLocks.add("L1:8");
 
     emb = cemb(nol("a", "L1", ke11));
     runTest(metaCache, ml, emb, "w");
-    
+
     activeLocks.add("L2:9");
-    
+
     emb = cemb(nol("a", "L1", ke11), nol("w", "L2", ke12));
     runTest(metaCache, ml, emb);
-    
+
     List<Range> ranges = nrl(new Range("a"), nr("b", "o"), nr("r", "z"));
     Map<String,Map<KeyExtent,List<Range>>> expected = createExpectedBinnings("L1", nol(ke11, nrl(new Range("a"), nr("b", "o"))), "L2",
         nol(ke12, nrl(nr("b", "o"), nr("r", "z"))));
-    
+
     runTest(null, ranges, metaCache, expected);
-    
+
     activeLocks.remove("L2:9");
-    
+
     expected = createExpectedBinnings("L1", nol(ke11, nrl(new Range("a"))));
     runTest(null, ranges, metaCache, expected, nrl(nr("b", "o"), nr("r", "z")));
-    
+
     activeLocks.clear();
-    
+
     expected = createExpectedBinnings();
     runTest(null, ranges, metaCache, expected, nrl(new Range("a"), nr("b", "o"), nr("r", "z")));
-    
+
     clearLocation(tservers, "tserver2", MTE, ke11, "8");
     clearLocation(tservers, "tserver2", MTE, ke12, "9");
     setLocation(tservers, "tserver2", MTE, ke11, "L3", "10");
     setLocation(tservers, "tserver2", MTE, ke12, "L4", "11");
-    
+
     runTest(null, ranges, metaCache, expected, nrl(new Range("a"), nr("b", "o"), nr("r", "z")));
-    
+
     activeLocks.add("L3:10");
 
     expected = createExpectedBinnings("L3", nol(ke11, nrl(new Range("a"))));
     runTest(null, ranges, metaCache, expected, nrl(nr("b", "o"), nr("r", "z")));
-    
+
     activeLocks.add("L4:11");
-    
+
     expected = createExpectedBinnings("L3", nol(ke11, nrl(new Range("a"), nr("b", "o"))), "L4", nol(ke12, nrl(nr("b", "o"), nr("r", "z"))));
     runTest(null, ranges, metaCache, expected);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/de8ecd4a/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 00714f9..0e54d5e 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
@@ -52,134 +52,135 @@ import org.apache.log4j.Logger;
  * 
  */
 public class HdfsZooInstance implements Instance {
-  
+
   public static class AccumuloNotInitializedException extends RuntimeException {
     private static final long serialVersionUID = 1L;
-    
+
     public AccumuloNotInitializedException(String string) {
       super(string);
     }
   }
-  
+
   private HdfsZooInstance() {
     AccumuloConfiguration acuConf = ServerConfiguration.getSiteConfiguration();
     zooCache = new ZooCache(acuConf.get(Property.INSTANCE_ZK_HOST), (int) acuConf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT));
   }
-  
+
   private static HdfsZooInstance cachedHdfsZooInstance = null;
-  
+
   public static synchronized Instance getInstance() {
     if (cachedHdfsZooInstance == null)
       cachedHdfsZooInstance = new HdfsZooInstance();
     return cachedHdfsZooInstance;
   }
-  
+
   private static ZooCache zooCache;
   private static String instanceId = null;
   private static final Logger log = Logger.getLogger(HdfsZooInstance.class);
-  
+
   @Override
   public String getRootTabletLocation() {
     String zRootLocPath = ZooUtil.getRoot(this) + RootTable.ZROOT_TABLET_LOCATION;
-    
+
     OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up root tablet location in zoocache.");
-    
+
     byte[] loc = zooCache.get(zRootLocPath);
-    
+
     opTimer.stop("Found root tablet at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
-    
+
     if (loc == null) {
       return null;
     }
-    
+
     return new String(loc).split("\\|")[0];
   }
-  
+
   @Override
   public List<String> getMasterLocations() {
-    
+
     String masterLocPath = ZooUtil.getRoot(this) + Constants.ZMASTER_LOCK;
-    
+
     OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up master location in zoocache.");
-    
+
     byte[] loc = ZooLock.getLockData(zooCache, masterLocPath, null);
-    
+
     opTimer.stop("Found master at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
-    
+
     if (loc == null) {
       return Collections.emptyList();
     }
-    
+
     return Collections.singletonList(new String(loc));
   }
-  
+
   @Override
   public String getInstanceID() {
     if (instanceId == null)
       _getInstanceID();
     return instanceId;
   }
-  
+
   private static synchronized void _getInstanceID() {
     if (instanceId == null) {
       String instanceIdFromFile = ZooUtil.getInstanceIDFromHdfs(ServerConstants.getInstanceIdLocation());
       instanceId = instanceIdFromFile;
     }
   }
-  
+
   @Override
   public String getInstanceName() {
     return ZooKeeperInstance.lookupInstanceName(zooCache, UUID.fromString(getInstanceID()));
   }
-  
+
   @Override
   public String getZooKeepers() {
     return ServerConfiguration.getSiteConfiguration().get(Property.INSTANCE_ZK_HOST);
   }
-  
+
   @Override
   public int getZooKeepersSessionTimeOut() {
     return (int) ServerConfiguration.getSiteConfiguration().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
   }
-  
+
   @Override
   public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException {
     return new ConnectorImpl(this, new Credentials(principal, token));
   }
-  
+
   @Deprecated
   @Override
   public Connector getConnector(String user, byte[] pass) throws AccumuloException, AccumuloSecurityException {
     return getConnector(user, new PasswordToken(pass));
   }
-  
+
   @Deprecated
   @Override
   public Connector getConnector(String user, ByteBuffer pass) throws AccumuloException, AccumuloSecurityException {
     return getConnector(user, ByteBufferUtil.toBytes(pass));
   }
-  
+
   @Deprecated
   @Override
   public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException {
     return getConnector(user, TextUtil.getBytes(new Text(pass.toString())));
   }
-  
+
   private AccumuloConfiguration conf = null;
-  
+
+  @Deprecated
   @Override
   public AccumuloConfiguration getConfiguration() {
     if (conf == null)
       conf = new ServerConfiguration(this).getConfiguration();
     return conf;
   }
-  
+
   @Override
   @Deprecated
   public void setConfiguration(AccumuloConfiguration conf) {
     this.conf = conf;
   }
-  
+
   public static void main(String[] args) {
     Instance instance = HdfsZooInstance.getInstance();
     System.out.println("Instance Name: " + instance.getInstanceName());
@@ -196,5 +197,5 @@ public class HdfsZooInstance implements Instance {
       throw new AccumuloException("Issues closing ZooKeeper, try again");
     }
   }
-  
+
 }


[3/6] git commit: ACCUMULO-1866 avoid using ServerConfigurationUtil in server code

Posted by ct...@apache.org.
ACCUMULO-1866 avoid using ServerConfigurationUtil in server code


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

Branch: refs/heads/master
Commit: 32b5aee44f4b6b1bdd4b531132c6f855f433bcc8
Parents: 5bd68ef
Author: Keith Turner <kt...@apache.org>
Authored: Thu Nov 21 16:01:30 2013 -0500
Committer: Keith Turner <kt...@apache.org>
Committed: Thu Nov 21 16:01:30 2013 -0500

----------------------------------------------------------------------
 .../apache/accumulo/server/client/BulkImporter.java    | 11 ++++++-----
 .../accumulo/gc/GarbageCollectWriteAheadLogs.java      |  4 ++--
 .../org/apache/accumulo/gc/SimpleGarbageCollector.java | 13 ++++++-------
 .../main/java/org/apache/accumulo/master/Master.java   |  4 ++--
 4 files changed, 16 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/32b5aee4/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java b/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
index 07aa1f8..8f35b23 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
@@ -36,7 +36,6 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.ServerClient;
-import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
 import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.client.impl.Translator;
@@ -61,6 +60,7 @@ import org.apache.accumulo.core.util.NamingThreadFactory;
 import org.apache.accumulo.core.util.StopWatch;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.trace.instrument.TraceRunnable;
@@ -145,7 +145,7 @@ public class BulkImporter {
           public void run() {
             List<TabletLocation> tabletsToAssignMapFileTo = Collections.emptyList();
             try {
-              tabletsToAssignMapFileTo = findOverlappingTablets(ServerConfigurationUtil.getConfiguration(instance), fs, locator, mapFile, credentials);
+              tabletsToAssignMapFileTo = findOverlappingTablets(ServerConfiguration.getSystemConfiguration(instance), fs, locator, mapFile, credentials);
             } catch (Exception ex) {
               log.warn("Unable to find tablets that overlap file " + mapFile.toString());
             }
@@ -208,7 +208,8 @@ public class BulkImporter {
             
             try {
               timer.start(Timers.QUERY_METADATA);
-              tabletsToAssignMapFileTo.addAll(findOverlappingTablets(ServerConfigurationUtil.getConfiguration(instance), fs, locator, entry.getKey(), ke, credentials));
+              tabletsToAssignMapFileTo.addAll(findOverlappingTablets(ServerConfiguration.getSystemConfiguration(instance), fs, locator, entry.getKey(), ke,
+                  credentials));
               timer.stop(Timers.QUERY_METADATA);
               keListIter.remove();
             } catch (Exception ex) {
@@ -585,8 +586,8 @@ public class BulkImporter {
   private List<KeyExtent> assignMapFiles(Credentials credentials, String location, Map<KeyExtent,List<PathSize>> assignmentsPerTablet)
       throws AccumuloException, AccumuloSecurityException {
     try {
-      long timeInMillis = ServerConfigurationUtil.getConfiguration(instance).getTimeInMillis(Property.TSERV_BULK_TIMEOUT);
-      TabletClientService.Iface client = ThriftUtil.getTServerClient(location, ServerConfigurationUtil.getConfiguration(instance), timeInMillis);
+      long timeInMillis = ServerConfiguration.getSystemConfiguration(instance).getTimeInMillis(Property.TSERV_BULK_TIMEOUT);
+      TabletClientService.Iface client = ThriftUtil.getTServerClient(location, ServerConfiguration.getSystemConfiguration(instance), timeInMillis);
       try {
         HashMap<KeyExtent,Map<String,org.apache.accumulo.core.data.thrift.MapFileInfo>> files = new HashMap<KeyExtent,Map<String,org.apache.accumulo.core.data.thrift.MapFileInfo>>();
         for (Entry<KeyExtent,List<PathSize>> entry : assignmentsPerTablet.entrySet()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/32b5aee4/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
----------------------------------------------------------------------
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
index 6afd42d..b311b77 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
@@ -30,7 +30,6 @@ import java.util.UUID;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.gc.thrift.GCStatus;
 import org.apache.accumulo.core.gc.thrift.GcCycleStats;
@@ -41,6 +40,7 @@ import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.ThriftUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.server.ServerConstants;
+import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.accumulo.server.security.SystemCredentials;
@@ -134,7 +134,7 @@ public class GarbageCollectWriteAheadLogs {
   }
   
   private int removeFiles(Map<String,Path> nameToFileMap, Map<String,ArrayList<Path>> serverToFileMap, Map<String, Path> sortedWALogs, final GCStatus status) {
-    AccumuloConfiguration conf = ServerConfigurationUtil.getConfiguration(instance);
+    AccumuloConfiguration conf = ServerConfiguration.getSystemConfiguration(instance);
     for (Entry<String,ArrayList<Path>> entry : serverToFileMap.entrySet()) {
       if (entry.getKey().isEmpty()) {
         // old-style log entry, just remove it

http://git-wip-us.apache.org/repos/asf/accumulo/blob/32b5aee4/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
----------------------------------------------------------------------
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
index 1f1b28d..ca1fb03 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
@@ -41,7 +41,6 @@ import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -154,9 +153,9 @@ public class SimpleGarbageCollector implements Iface {
     this.credentials = credentials;
     this.instance = instance;
     
-    gcStartDelay = ServerConfigurationUtil.getConfiguration(instance).getTimeInMillis(Property.GC_CYCLE_START);
-    long gcDelay = ServerConfigurationUtil.getConfiguration(instance).getTimeInMillis(Property.GC_CYCLE_DELAY);
-    numDeleteThreads = ServerConfigurationUtil.getConfiguration(instance).getCount(Property.GC_DELETE_THREADS);
+    gcStartDelay = ServerConfiguration.getSystemConfiguration(instance).getTimeInMillis(Property.GC_CYCLE_START);
+    long gcDelay = ServerConfiguration.getSystemConfiguration(instance).getTimeInMillis(Property.GC_CYCLE_DELAY);
+    numDeleteThreads = ServerConfiguration.getSystemConfiguration(instance).getCount(Property.GC_DELETE_THREADS);
     log.info("start delay: " + gcStartDelay + " milliseconds");
     log.info("time delay: " + gcDelay + " milliseconds");
     log.info("safemode: " + opts.safeMode);
@@ -482,7 +481,7 @@ public class SimpleGarbageCollector implements Iface {
       
       Trace.offNoFlush();
       try {
-        long gcDelay = ServerConfigurationUtil.getConfiguration(instance).getTimeInMillis(Property.GC_CYCLE_DELAY);
+        long gcDelay = ServerConfiguration.getSystemConfiguration(instance).getTimeInMillis(Property.GC_CYCLE_DELAY);
         log.debug("Sleeping for " + gcDelay + " milliseconds");
         Thread.sleep(gcDelay);
       } catch (InterruptedException e) {
@@ -535,8 +534,8 @@ public class SimpleGarbageCollector implements Iface {
   
   private HostAndPort startStatsService() throws UnknownHostException {
     Processor<Iface> processor = new Processor<Iface>(TraceWrap.service(this));
-    int port = ServerConfigurationUtil.getConfiguration(instance).getPort(Property.GC_PORT);
-    long maxMessageSize = ServerConfigurationUtil.getConfiguration(instance).getMemoryInBytes(Property.GENERAL_MAX_MESSAGE_SIZE);
+    int port = ServerConfiguration.getSystemConfiguration(instance).getPort(Property.GC_PORT);
+    long maxMessageSize = ServerConfiguration.getSystemConfiguration(instance).getMemoryInBytes(Property.GENERAL_MAX_MESSAGE_SIZE);
     HostAndPort result = HostAndPort.fromParts(opts.getAddress(), port);
     try {
       port = TServerUtils.startTServer(result, processor, this.getClass().getSimpleName(), "GC Monitor Service", 2, 1000, maxMessageSize).address.getPort();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/32b5aee4/server/master/src/main/java/org/apache/accumulo/master/Master.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/Master.java b/server/master/src/main/java/org/apache/accumulo/master/Master.java
index e17bccc..0786243 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/Master.java
@@ -44,7 +44,6 @@ import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TableOperationsImpl;
-import org.apache.accumulo.core.client.impl.ServerConfigurationUtil;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.ThriftTransportPool;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
@@ -787,7 +786,8 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
 
     private void updatePlugins(String property) {
       if (property.equals(Property.MASTER_TABLET_BALANCER.getKey())) {
-        TabletBalancer balancer = ServerConfigurationUtil.getConfiguration(instance).instantiateClassProperty(Property.MASTER_TABLET_BALANCER, TabletBalancer.class,
+        TabletBalancer balancer = ServerConfiguration.getSystemConfiguration(instance).instantiateClassProperty(Property.MASTER_TABLET_BALANCER,
+            TabletBalancer.class,
             new DefaultLoadBalancer());
         balancer.init(serverConfig);
         tabletBalancer = balancer;


[5/6] git commit: ACCUMULO-1852 Fix native maps on OS X 10.9

Posted by ct...@apache.org.
ACCUMULO-1852 Fix native maps on OS X 10.9


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

Branch: refs/heads/master
Commit: 2aad7fe0af7701609df67dea3daf872a526761be
Parents: de8ecd4
Author: Christopher Tubbs <ct...@apache.org>
Authored: Thu Nov 21 17:08:23 2013 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Thu Nov 21 17:26:03 2013 -0500

----------------------------------------------------------------------
 server/native/src/main/c++/nativeMap/BlockAllocator.h |  1 -
 server/native/src/main/resources/Makefile             | 11 ++++++-----
 2 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/2aad7fe0/server/native/src/main/c++/nativeMap/BlockAllocator.h
----------------------------------------------------------------------
diff --git a/server/native/src/main/c++/nativeMap/BlockAllocator.h b/server/native/src/main/c++/nativeMap/BlockAllocator.h
index d62ffd5..81c14d8 100644
--- a/server/native/src/main/c++/nativeMap/BlockAllocator.h
+++ b/server/native/src/main/c++/nativeMap/BlockAllocator.h
@@ -20,7 +20,6 @@
 #define _BLOCK_ALLOCATOR_H_ 1
 
 #include <new>
-#include <bits/functexcept.h>
 #include <iostream>
 #include <string>
 #include <vector>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2aad7fe0/server/native/src/main/resources/Makefile
----------------------------------------------------------------------
diff --git a/server/native/src/main/resources/Makefile b/server/native/src/main/resources/Makefile
index 9f0ecde..c9bdc91 100644
--- a/server/native/src/main/resources/Makefile
+++ b/server/native/src/main/resources/Makefile
@@ -17,6 +17,7 @@ SRCS=$(wildcard nativeMap/*.cc)
 HDRS=$(wildcard nativeMap/*.h) $(wildcard javah/*.h)
 TESTSRCS=$(wildcard testNativeMap/*.cc)
 CXX=g++
+MAVERICKFLAGS=
 
 ifeq ($(shell uname),Linux)
 	JAVA_HOME=$(shell dirname $$(dirname $$(readlink -ef $$(which javah))))
@@ -27,10 +28,10 @@ endif
 ifeq ($(shell uname),Darwin)
 	JAVA_HOME=$(shell /usr/libexec/java_home)
 	NATIVE_LIB:= libaccumulo.dylib
-	# Update flags for OSX-10.9 and Xcode 5.0.1
-	# I think that we should be able to remove the `-undefined dynamic_lookup` option,
-	# but I don't know exactly how to go about this.
-	CXXFLAGS=-m64 -dynamiclib -undefined dynamic_lookup -O3 -I/System/Library/Frameworks/JavaVM.framework/Headers -I$(JAVA_HOME)/include -I$(JAVA_HOME)/include/darwin -I/usr/include/c++/4.2.1 -Ijavah 
+ifneq (,$(findstring 10.9,$(shell sw_vers -productVersion)))
+	MAVERICKFLAGS=-stdlib=libstdc++
+endif
+	CXXFLAGS=-m64 -dynamiclib -undefined dynamic_lookup -O3 -I/System/Library/Frameworks/JavaVM.framework/Headers -I$(JAVA_HOME)/include -I$(JAVA_HOME)/include/darwin -Ijavah $(MAVERICK_FLAGS)
 endif
 
 all : $(NATIVE_LIB)
@@ -44,7 +45,7 @@ testJavaHome :
 	@echo JAVA_HOME is $(JAVA_HOME)
 
 runTests : $(NATIVE_LIB) $(TESTSRCS)
-	$(CXX) -g -Wall -I/System/Library/Frameworks/JavaVM.framework/Headers -I$(JAVA_HOME)/include -I$(JAVA_HOME)/include/linux -InativeMap -o $@ $(TESTSRCS) $(NATIVE_LIB)
+	$(CXX) -g -Wall -I/System/Library/Frameworks/JavaVM.framework/Headers -I$(JAVA_HOME)/include -I$(JAVA_HOME)/include/linux -InativeMap -o $@ $(TESTSRCS) $(NATIVE_LIB) $(MAVERICK_FLAGS)
 	LD_LIBRARY_PATH=./ ./$@ 20 20 20 20 20 20 20 20 true
 
 clean :


[6/6] git commit: Merge branch '1.6.0-SNAPSHOT'

Posted by ct...@apache.org.
Merge branch '1.6.0-SNAPSHOT'


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

Branch: refs/heads/master
Commit: 0d802767099a2e485a826de9e1a9b1c38a22e7c0
Parents: 9b41155 2aad7fe
Author: Christopher Tubbs <ct...@apache.org>
Authored: Thu Nov 21 17:41:01 2013 -0500
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Thu Nov 21 17:41:01 2013 -0500

----------------------------------------------------------------------
 .../apache/accumulo/core/cli/ClientOpts.java    |   4 +-
 .../core/client/ClientConfiguration.java        | 260 +++++++
 .../apache/accumulo/core/client/Instance.java   |  26 +-
 .../accumulo/core/client/ZooKeeperInstance.java |  19 +-
 .../client/admin/InstanceOperationsImpl.java    |   8 +-
 .../core/client/admin/TableOperationsImpl.java  |   6 +-
 .../core/client/impl/ConditionalWriterImpl.java |   6 +-
 .../accumulo/core/client/impl/MasterClient.java |   2 +-
 .../core/client/impl/OfflineScanner.java        |   4 +-
 .../core/client/impl/ScannerIterator.java       |   2 +-
 .../accumulo/core/client/impl/ServerClient.java |   4 +-
 .../client/impl/ServerConfigurationFactory.java |  30 -
 .../client/impl/ServerConfigurationUtil.java    |  63 ++
 .../impl/TabletServerBatchReaderIterator.java   |   2 +-
 .../client/impl/TabletServerBatchWriter.java    |   6 +-
 .../accumulo/core/client/impl/Writer.java       |   2 +-
 .../core/client/mapred/AbstractInputFormat.java |   2 +-
 .../core/client/mapred/AccumuloInputFormat.java |   2 +-
 .../mapred/AccumuloMultiTableInputFormat.java   |   2 +-
 .../client/mapred/AccumuloOutputFormat.java     |   2 +-
 .../client/mapred/AccumuloRowInputFormat.java   |   2 +-
 .../client/mapreduce/AbstractInputFormat.java   |   2 +-
 .../client/mapreduce/AccumuloInputFormat.java   |   2 +-
 .../AccumuloMultiTableInputFormat.java          |   2 +-
 .../client/mapreduce/AccumuloOutputFormat.java  |   2 +-
 .../mapreduce/AccumuloRowInputFormat.java       |   2 +-
 .../mapreduce/lib/util/ConfiguratorBase.java    |   2 +-
 .../accumulo/core/client/mock/MockInstance.java |  37 +-
 .../accumulo/core/conf/ClientConfiguration.java | 310 --------
 .../core/metadata/MetadataLocationObtainer.java |   8 +-
 .../apache/accumulo/core/util/shell/Shell.java  |  11 +-
 .../core/util/shell/ShellOptionsJC.java         |   4 +-
 .../core/client/impl/TabletLocatorImplTest.java | 755 ++++++++++---------
 .../lib/util/ConfiguratorBaseTest.java          |   4 +-
 .../core/conf/ClientConfigurationTest.java      |   3 +-
 .../core/util/shell/ShellSetInstanceTest.java   |  17 +-
 .../examples/simple/filedata/FileDataQuery.java |   2 +-
 .../simple/mapreduce/TokenFileWordCount.java    |   2 +-
 .../examples/simple/reservations/ARS.java       |   2 +-
 .../minicluster/MiniAccumuloCluster.java        |   2 +-
 .../minicluster/MiniAccumuloInstance.java       |   2 +-
 .../org/apache/accumulo/proxy/ProxyServer.java  |   2 +-
 .../accumulo/server/client/BulkImporter.java    |  11 +-
 .../accumulo/server/client/HdfsZooInstance.java |  67 +-
 .../accumulo/utils/metanalysis/IndexMeta.java   |   2 +-
 .../gc/GarbageCollectWriteAheadLogs.java        |   4 +-
 .../accumulo/gc/SimpleGarbageCollector.java     |  13 +-
 .../java/org/apache/accumulo/master/Master.java |   4 +-
 .../src/main/c++/nativeMap/BlockAllocator.h     |   1 -
 server/native/src/main/resources/Makefile       |  11 +-
 .../apache/accumulo/test/IMMLGBenchmark.java    |   2 +-
 .../metadata/MetadataBatchScanTest.java         |   2 +-
 .../test/performance/thrift/NullTserver.java    |   2 +-
 .../apache/accumulo/test/randomwalk/State.java  |   2 +-
 .../test/randomwalk/multitable/CopyTool.java    |   2 +-
 .../randomwalk/sequential/MapRedVerifyTool.java |   2 +-
 .../accumulo/test/scalability/ScaleTest.java    |   2 +-
 .../test/MultiTableBatchWriterTest.java         |   2 +-
 58 files changed, 873 insertions(+), 883 deletions(-)
----------------------------------------------------------------------