You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2014/09/05 23:17:32 UTC

[16/18] git commit: ACCUMULO-1957 added Durability options to the Proxy and Shell, swap test and proxy module dependencies

ACCUMULO-1957 added Durability options to the Proxy and Shell, swap test and proxy module dependencies


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

Branch: refs/heads/master
Commit: 3a0beab0d785f607b8f2cc73392467a830528b7f
Parents: f5b598e
Author: Eric C. Newton <er...@gmail.com>
Authored: Thu Sep 4 14:05:42 2014 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Sep 5 17:16:59 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/client/BatchWriterConfig.java |    7 +-
 .../apache/accumulo/core/client/Durability.java |   56 +-
 .../core/client/impl/ConditionalWriterImpl.java |    2 +-
 .../core/client/impl/DurabilityImpl.java        |   68 +
 .../client/impl/TabletServerBatchWriter.java    |    4 +-
 .../org/apache/accumulo/core/conf/Property.java |    6 +-
 .../core/client/BatchWriterConfigTest.java      |   16 +
 .../simple/client/ReadWriteExample.java         |   35 +-
 proxy/pom.xml                                   |    5 -
 proxy/src/main/cpp/AccumuloProxy.cpp            | 1454 ++++++++--------
 proxy/src/main/cpp/proxy_types.cpp              |  186 ++-
 proxy/src/main/cpp/proxy_types.h                |   50 +-
 .../org/apache/accumulo/proxy/ProxyServer.java  |   22 +
 .../proxy/thrift/ConditionalWriterOptions.java  |  126 +-
 .../accumulo/proxy/thrift/Durability.java       |   70 +
 .../accumulo/proxy/thrift/WriterOptions.java    |  125 +-
 proxy/src/main/python/ttypes.py                 |   51 +-
 proxy/src/main/ruby/proxy_types.rb              |   24 +-
 proxy/src/main/thrift/proxy.thrift              |   13 +
 .../apache/accumulo/proxy/SimpleProxyIT.java    | 1575 ------------------
 .../proxy/TestProxyInstanceOperations.java      |   83 -
 .../accumulo/proxy/TestProxyReadWrite.java      |  478 ------
 .../proxy/TestProxySecurityOperations.java      |  157 --
 .../proxy/TestProxyTableOperations.java         |  212 ---
 .../java/org/apache/accumulo/master/Master.java |   35 +
 .../accumulo/tserver/TabletMutations.java       |    4 +-
 .../apache/accumulo/tserver/TabletServer.java   |   34 +-
 .../tserver/TabletServerResourceManager.java    |    7 +-
 .../apache/accumulo/tserver/log/DfsLogger.java  |   19 +-
 .../tserver/log/TabletServerLogger.java         |    8 +-
 .../apache/accumulo/tserver/tablet/Tablet.java  |    4 +-
 .../accumulo/tserver/log/DfsLoggerTest.java     |   59 +
 .../accumulo/shell/commands/InsertCommand.java  |   27 +-
 test/pom.xml                                    |    4 +
 .../apache/accumulo/test/WrongTabletTest.java   |    8 +-
 .../test/performance/thrift/NullTserver.java    |   92 +-
 .../test/randomwalk/concurrent/Config.java      |    2 +-
 .../accumulo/proxy/ProxyDurabilityIT.java       |  141 ++
 .../apache/accumulo/proxy/SimpleProxyIT.java    | 1575 ++++++++++++++++++
 .../proxy/TestProxyInstanceOperations.java      |   83 +
 .../accumulo/proxy/TestProxyReadWrite.java      |  478 ++++++
 .../proxy/TestProxySecurityOperations.java      |  157 ++
 .../proxy/TestProxyTableOperations.java         |  212 +++
 .../accumulo/test/functional/BloomFilterIT.java |    2 +-
 .../accumulo/test/functional/DurabilityIT.java  |    3 +-
 45 files changed, 4296 insertions(+), 3483 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java b/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
index 270a89e..23ab39e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/BatchWriterConfig.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.accumulo.core.client.impl.DurabilityImpl;
 import org.apache.commons.lang.builder.HashCodeBuilder;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.util.StringUtils;
@@ -178,6 +179,10 @@ public class BatchWriterConfig implements Writable {
   }
 
   /**
+   * Change the durability for the BatchWriter session. The default durability is "default" which 
+   * is the table's durability setting.  If the durability is set to something other than the default,
+   * it will override the durability setting of the table.
+   * 
    * @param durability the Durability to be used by the BatchWriter
    * @since 1.7.0
    *
@@ -241,7 +246,7 @@ public class BatchWriterConfig implements Writable {
       } else if ("timeout".equals(key)) {
         timeout = Long.valueOf(value);
       } else if ("durability".equals(key)) {
-        durability = Durability.fromString(value);
+        durability = DurabilityImpl.fromString(value);
       } else {
         /* ignore any other properties */
       }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/core/src/main/java/org/apache/accumulo/core/client/Durability.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Durability.java b/core/src/main/java/org/apache/accumulo/core/client/Durability.java
index 6de666d..3e69cb2 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Durability.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Durability.java
@@ -16,8 +16,6 @@
  */
 package org.apache.accumulo.core.client;
 
-import org.apache.accumulo.core.tabletserver.thrift.TDurability;
-
 /**
  * The value for the durability of a BatchWriter or ConditionalWriter.
  * @since 1.7.0
@@ -43,57 +41,5 @@ public enum Durability {
   /**
    * Write mutations to the write-ahead log, and ensure the data is saved to persistent storage.
    */
-  SYNC;
-
-  // for internal use only
-  public TDurability toThrift() {
-    switch (this) {
-      case DEFAULT:
-        return TDurability.DEFAULT;
-      case SYNC:
-        return TDurability.SYNC;
-      case FLUSH:
-        return TDurability.FLUSH;
-      case LOG:
-        return TDurability.LOG;
-      default:
-        return TDurability.NONE;
-    }
-  }
-
-  // for internal use only
-  static public Durability fromString(String value) {
-    try {
-      return Durability.valueOf(value.toUpperCase());
-    } catch (IllegalArgumentException ex) {
-      return Durability.SYNC;
-    }
-  }
-
-  // for internal use only
-  public static Durability fromThrift(TDurability tdurabilty) {
-    if (tdurabilty == null) {
-      return Durability.DEFAULT;
-    }
-    switch (tdurabilty) {
-      case DEFAULT:
-        return Durability.DEFAULT;
-      case SYNC:
-        return Durability.SYNC;
-      case FLUSH:
-        return Durability.FLUSH;
-      case LOG:
-        return Durability.LOG;
-      default:
-        return Durability.NONE;
-    }
-  }
-
-  // for internal use only
-  public Durability resolveDurability(Durability tabletDurability) {
-    if (this == Durability.DEFAULT) {
-      return tabletDurability;
-    }
-    return this;
-  }
+  SYNC
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/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 02954fa..e8af187 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
@@ -500,7 +500,7 @@ class ConditionalWriterImpl implements ConditionalWriter {
     }
     
     TConditionalSession tcs = client.startConditionalUpdate(tinfo, credentials.toThrift(instance), ByteBufferUtil.toByteBuffers(auths.getAuthorizations()),
-        tableId, durability.toThrift());
+        tableId, DurabilityImpl.toThrift(durability));
     
     synchronized (cachedSessionIDs) {
       SessionID sid = new SessionID();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/core/src/main/java/org/apache/accumulo/core/client/impl/DurabilityImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/DurabilityImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/DurabilityImpl.java
new file mode 100644
index 0000000..b2a0a98
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/DurabilityImpl.java
@@ -0,0 +1,68 @@
+/*
+ * 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.Durability;
+import org.apache.accumulo.core.tabletserver.thrift.TDurability;
+
+public class DurabilityImpl {
+
+  public static TDurability toThrift(Durability durability) {
+    switch (durability) {
+      case DEFAULT:
+        return TDurability.DEFAULT;
+      case SYNC:
+        return TDurability.SYNC;
+      case FLUSH:
+        return TDurability.FLUSH;
+      case LOG:
+        return TDurability.LOG;
+      default:
+        return TDurability.NONE;
+    }
+  }
+
+  public static Durability fromString(String value) {
+    return Durability.valueOf(value.toUpperCase());
+  }
+
+  public static Durability fromThrift(TDurability tdurabilty) {
+    if (tdurabilty == null) {
+      return Durability.DEFAULT;
+    }
+    switch (tdurabilty) {
+      case DEFAULT:
+        return Durability.DEFAULT;
+      case SYNC:
+        return Durability.SYNC;
+      case FLUSH:
+        return Durability.FLUSH;
+      case LOG:
+        return Durability.LOG;
+      default:
+        return Durability.NONE;
+    }
+  }
+
+  public static Durability resolveDurabilty(Durability durability, Durability tabletDurability) {
+    if (durability == Durability.DEFAULT) {
+      return tabletDurability;
+    }
+    return durability;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/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 c68f842..5eec397 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
@@ -860,7 +860,7 @@ public class TabletServerBatchWriter {
             Entry<KeyExtent,List<Mutation>> entry = tabMuts.entrySet().iterator().next();
             
             try {
-              client.update(tinfo, credentials.toThrift(instance), entry.getKey().toThrift(), entry.getValue().get(0).toThrift(), durability.toThrift());
+              client.update(tinfo, credentials.toThrift(instance), entry.getKey().toThrift(), entry.getValue().get(0).toThrift(), DurabilityImpl.toThrift(durability));
             } catch (NotServingTabletException e) {
               allFailures.addAll(entry.getKey().getTableId().toString(), entry.getValue());
               TabletLocator.getLocator(instance, new Text(entry.getKey().getTableId())).invalidateCache(entry.getKey());
@@ -870,7 +870,7 @@ public class TabletServerBatchWriter {
             timeoutTracker.madeProgress();
           } else {
             
-            long usid = client.startUpdate(tinfo, credentials.toThrift(instance), durability.toThrift());
+            long usid = client.startUpdate(tinfo, credentials.toThrift(instance), DurabilityImpl.toThrift(durability));
             
             List<TMutation> updates = new ArrayList<TMutation>();
             for (Entry<KeyExtent,List<Mutation>> entry : tabMuts.entrySet()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 6ac6fa9..35cd0a6 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -197,11 +197,15 @@ public enum Property {
   TSERV_INDEXCACHE_SIZE("tserver.cache.index.size", "512M", PropertyType.MEMORY, "Specifies the size of the cache for file indices."),
   TSERV_PORTSEARCH("tserver.port.search", "false", PropertyType.BOOLEAN, "if the ports above are in use, search higher ports until one is available"),
   TSERV_CLIENTPORT("tserver.port.client", "9997", PropertyType.PORT, "The port used for handling client connections on the tablet servers"),
+  @Deprecated
   TSERV_MUTATION_QUEUE_MAX("tserver.mutation.queue.max", "1M", PropertyType.MEMORY,
-      "The amount of memory to use to store write-ahead-log mutations-per-session before flushing them. Since the buffer is per write session, consider the"
+      "This setting is deprecated. See tserver.total.mutation.queue.max. " 
+          + "The amount of memory to use to store write-ahead-log mutations-per-session before flushing them. Since the buffer is per write session, consider the"
           + " max number of concurrent writer when configuring. When using Hadoop 2, Accumulo will call hsync() on the WAL . For a small number of "
           + "concurrent writers, increasing this buffer size decreases the frequncy of hsync calls. For a large number of concurrent writers a small buffers "
           + "size is ok because of group commit."),
+  TSERV_TOTAL_MUTATION_QUEUE_MAX("tserver.total.mutation.queue.max", "50M", PropertyType.MEMORY, 
+      "The amount of memory used to store write-ahead-log mutations before flushing them."),
   TSERV_TABLET_SPLIT_FINDMIDPOINT_MAXOPEN("tserver.tablet.split.midpoint.files.max", "30", PropertyType.COUNT,
       "To find a tablets split points, all index files are opened. This setting determines how many index "
           + "files can be opened at once. When there are more index files than this setting multiple passes "

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/core/src/test/java/org/apache/accumulo/core/client/BatchWriterConfigTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/BatchWriterConfigTest.java b/core/src/test/java/org/apache/accumulo/core/client/BatchWriterConfigTest.java
index 231afe5..8f0812c 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/BatchWriterConfigTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/BatchWriterConfigTest.java
@@ -40,12 +40,14 @@ public class BatchWriterConfigTest {
     long expectedMaxLatency = 120000l;
     long expectedTimeout = Long.MAX_VALUE;
     int expectedMaxWriteThreads = 3;
+    Durability expectedDurability = Durability.DEFAULT;
     
     BatchWriterConfig defaults = new BatchWriterConfig();
     assertEquals(expectedMaxMemory, defaults.getMaxMemory());
     assertEquals(expectedMaxLatency, defaults.getMaxLatency(TimeUnit.MILLISECONDS));
     assertEquals(expectedTimeout, defaults.getTimeout(TimeUnit.MILLISECONDS));
     assertEquals(expectedMaxWriteThreads, defaults.getMaxWriteThreads());
+    assertEquals(expectedDurability, defaults.getDurability());
   }
   
   @Test
@@ -55,11 +57,13 @@ public class BatchWriterConfigTest {
     bwConfig.setMaxLatency(22, TimeUnit.HOURS);
     bwConfig.setTimeout(33, TimeUnit.DAYS);
     bwConfig.setMaxWriteThreads(42);
+    bwConfig.setDurability(Durability.NONE);
     
     assertEquals(1123581321l, bwConfig.getMaxMemory());
     assertEquals(22 * 60 * 60 * 1000l, bwConfig.getMaxLatency(TimeUnit.MILLISECONDS));
     assertEquals(33 * 24 * 60 * 60 * 1000l, bwConfig.getTimeout(TimeUnit.MILLISECONDS));
     assertEquals(42, bwConfig.getMaxWriteThreads());
+    assertEquals(Durability.NONE, bwConfig.getDurability());
   }
   
   @Test
@@ -133,6 +137,7 @@ public class BatchWriterConfigTest {
     assertNotEquals(9898989l, bwDefaults.getTimeout(TimeUnit.MILLISECONDS));
     assertNotEquals(42, bwDefaults.getMaxWriteThreads());
     assertNotEquals(1123581321l, bwDefaults.getMaxMemory());
+    assertNotEquals(Durability.FLUSH, bwDefaults.getDurability());
     
     // test setting all fields
     BatchWriterConfig bwConfig = new BatchWriterConfig();
@@ -140,6 +145,7 @@ public class BatchWriterConfigTest {
     bwConfig.setTimeout(9898989l, TimeUnit.MILLISECONDS);
     bwConfig.setMaxWriteThreads(42);
     bwConfig.setMaxMemory(1123581321l);
+    bwConfig.setDurability(Durability.FLUSH);
     byte[] bytes = createBytes(bwConfig);
     checkBytes(bwConfig, bytes);
     
@@ -157,6 +163,12 @@ public class BatchWriterConfigTest {
     bytes = createBytes(bwConfig);
     assertEquals("     v#maxWriteThreads=24,timeout=3000", new String(bytes, StandardCharsets.UTF_8));
     checkBytes(bwConfig, bytes);
+    
+    // test human-readable durability
+    bwConfig = new BatchWriterConfig();
+    bwConfig.setDurability(Durability.LOG);
+    bytes = createBytes(bwConfig);
+    assertEquals("     e#durability=LOG", new String(bytes, StandardCharsets.UTF_8));
   }
 
   @Test
@@ -166,6 +178,10 @@ public class BatchWriterConfigTest {
     assertEquals(cfg1.hashCode(), cfg2.hashCode());
     cfg2.setMaxMemory(1);
     assertNotEquals(cfg1, cfg2);
+    cfg2 = new BatchWriterConfig();
+    cfg2.setDurability(Durability.FLUSH);
+    assertNotEquals(cfg1, cfg2);
+    assertNotEquals(cfg1.hashCode(), cfg2.hashCode());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
----------------------------------------------------------------------
diff --git a/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java b/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
index 7bc44e8..2a42d29 100644
--- a/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
+++ b/examples/simple/src/main/java/org/apache/accumulo/examples/simple/client/ReadWriteExample.java
@@ -27,6 +27,7 @@ import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.impl.DurabilityImpl;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
@@ -42,16 +43,16 @@ public class ReadWriteExample {
   // defaults
   private static final String DEFAULT_AUTHS = "LEVEL1,GROUP1";
   private static final String DEFAULT_TABLE_NAME = "test";
-  
+
   private Connector conn;
-  
+
   static class DurabilityConverter implements IStringConverter<Durability> {
     @Override
     public Durability convert(String value) {
-      return Durability.fromString(value);
-    }    
+      return DurabilityImpl.fromString(value);
+    }
   }
-  
+
   static class Opts extends ClientOnDefaultTable {
     @Parameter(names = {"-C", "--createtable"}, description = "create table before doing anything")
     boolean createtable = false;
@@ -65,26 +66,26 @@ public class ReadWriteExample {
     boolean deleteEntries = false;
     @Parameter(names = {"--durability"}, description = "durabilty used for writes (none, log, flush or sync)", converter=DurabilityConverter.class)
     Durability durability = Durability.DEFAULT;
-    
+
     public Opts() {
       super(DEFAULT_TABLE_NAME);
       auths = new Authorizations(DEFAULT_AUTHS.split(","));
     }
   }
-  
+
   // hidden constructor
   private ReadWriteExample() {}
-  
+
   private void execute(Opts opts, ScannerOpts scanOpts) throws Exception {
     conn = opts.getConnector();
-    
+
     // add the authorizations to the user
     Authorizations userAuthorizations = conn.securityOperations().getUserAuthorizations(opts.principal);
     ByteArraySet auths = new ByteArraySet(userAuthorizations.getAuthorizations());
     auths.addAll(opts.auths.getAuthorizations());
     if (!auths.isEmpty())
       conn.securityOperations().changeUserAuthorizations(opts.principal, new Authorizations(auths));
-    
+
     // create table
     if (opts.createtable) {
       SortedSet<Text> partitionKeys = new TreeSet<Text>();
@@ -93,10 +94,10 @@ public class ReadWriteExample {
       conn.tableOperations().create(opts.getTableName());
       conn.tableOperations().addSplits(opts.getTableName(), partitionKeys);
     }
-    
+
     // send mutations
     createEntries(opts);
-    
+
     // read entries
     if (opts.readEntries) {
       // Note that the user needs to have the authorizations for the specified scan authorizations
@@ -106,24 +107,24 @@ public class ReadWriteExample {
       for (Entry<Key,Value> entry : scanner)
         System.out.println(entry.getKey().toString() + " -> " + entry.getValue().toString());
     }
-    
+
     // delete table
     if (opts.deletetable)
       conn.tableOperations().delete(opts.getTableName());
   }
-  
+
   private void createEntries(Opts opts) throws Exception {
     if (opts.createEntries || opts.deleteEntries) {
       BatchWriterConfig cfg = new BatchWriterConfig();
       cfg.setDurability(opts.durability);
       BatchWriter writer = conn.createBatchWriter(opts.getTableName(), cfg);
       ColumnVisibility cv = new ColumnVisibility(opts.auths.toString().replace(',', '|'));
-      
+
       Text cf = new Text("datatypes");
       Text cq = new Text("xml");
       byte[] row = {'h', 'e', 'l', 'l', 'o', '\0'};
       byte[] value = {'w', 'o', 'r', 'l', 'd', '\0'};
-      
+
       for (int i = 0; i < 10; i++) {
         row[row.length - 1] = (byte) i;
         Mutation m = new Mutation(new Text(row));
@@ -139,7 +140,7 @@ public class ReadWriteExample {
       writer.close();
     }
   }
-  
+
   public static void main(String[] args) throws Exception {
     ReadWriteExample rwe = new ReadWriteExample();
     Opts opts = new Opts();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3a0beab0/proxy/pom.xml
----------------------------------------------------------------------
diff --git a/proxy/pom.xml b/proxy/pom.xml
index 4f53450..9312d7b 100644
--- a/proxy/pom.xml
+++ b/proxy/pom.xml
@@ -77,11 +77,6 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.accumulo</groupId>
-      <artifactId>accumulo-test</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-log4j12</artifactId>
       <scope>test</scope>