http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java
 
b/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java
new file mode 100644
index 0000000..58f3777
--- /dev/null
+++ 
b/core/src/test/java/org/apache/accumulo/core/client/mock/MockTableOperationsTest.java
@@ -0,0 +1,345 @@
+/*
+ * 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.mock;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+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.BatchScanner;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+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.MutationsRejectedException;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.NewTableConfiguration;
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.file.FileOperations;
+import org.apache.accumulo.core.file.FileSKVWriter;
+import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.iterators.user.VersioningIterator;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+import com.google.common.collect.Iterators;
+
+@Deprecated
+public class MockTableOperationsTest {
+
+  @Rule
+  public TestName test = new TestName();
+
+  private Connector conn;
+
+  @Before
+  public void setupInstance() throws Exception {
+    Instance inst = new MockInstance(test.getMethodName());
+    conn = inst.getConnector("user", new PasswordToken("pass"));
+  }
+
+  @Test
+  public void testCreateUseVersions() throws AccumuloException, 
AccumuloSecurityException, TableExistsException, TableNotFoundException {
+    String t = "tableName1";
+
+    {
+      conn.tableOperations().create(t, new 
NewTableConfiguration().withoutDefaultIterators().setTimeType(TimeType.LOGICAL));
+
+      writeVersionable(conn, t, 3);
+      assertVersionable(conn, t, 3);
+
+      IteratorSetting settings = new IteratorSetting(20, 
VersioningIterator.class);
+      conn.tableOperations().attachIterator(t, settings);
+
+      assertVersionable(conn, t, 1);
+
+      conn.tableOperations().delete(t);
+    }
+
+    {
+      conn.tableOperations().create(t, new 
NewTableConfiguration().setTimeType(TimeType.MILLIS));
+
+      try {
+        IteratorSetting settings = new IteratorSetting(20, 
VersioningIterator.class);
+        conn.tableOperations().attachIterator(t, settings);
+        Assert.fail();
+      } catch (AccumuloException ex) {}
+
+      writeVersionable(conn, t, 3);
+      assertVersionable(conn, t, 1);
+
+      conn.tableOperations().delete(t);
+    }
+  }
+
+  protected void writeVersionable(Connector c, String tableName, int size) 
throws TableNotFoundException, MutationsRejectedException {
+    for (int i = 0; i < size; i++) {
+      BatchWriter w = c.createBatchWriter(tableName, new BatchWriterConfig());
+      Mutation m = new Mutation("row1");
+      m.put("cf", "cq", String.valueOf(i));
+      w.addMutation(m);
+      w.close();
+    }
+  }
+
+  protected void assertVersionable(Connector c, String tableName, int size) 
throws TableNotFoundException {
+    BatchScanner s = c.createBatchScanner(tableName, Authorizations.EMPTY, 1);
+    s.setRanges(Collections.singleton(Range.exact("row1", "cf", "cq")));
+    int count = 0;
+    for (Map.Entry<Key,Value> e : s) {
+      Assert.assertEquals("row1", e.getKey().getRow().toString());
+      Assert.assertEquals("cf", e.getKey().getColumnFamily().toString());
+      Assert.assertEquals("cq", e.getKey().getColumnQualifier().toString());
+      count++;
+
+    }
+    Assert.assertEquals(size, count);
+    s.close();
+  }
+
+  @Test
+  public void testTableNotFound() throws AccumuloException, 
AccumuloSecurityException, TableExistsException, TableNotFoundException {
+    IteratorSetting setting = new IteratorSetting(100, "myvers", 
VersioningIterator.class);
+    String t = "tableName";
+    try {
+      conn.tableOperations().attachIterator(t, setting);
+      Assert.fail();
+    } catch (TableNotFoundException e) {}
+    try {
+      conn.tableOperations().checkIteratorConflicts(t, setting, 
EnumSet.allOf(IteratorScope.class));
+      Assert.fail();
+    } catch (TableNotFoundException e) {}
+    try {
+      conn.tableOperations().delete(t);
+      Assert.fail();
+    } catch (TableNotFoundException e) {}
+    try {
+      conn.tableOperations().getIteratorSetting(t, "myvers", 
IteratorScope.scan);
+      Assert.fail();
+    } catch (TableNotFoundException e) {}
+    try {
+      conn.tableOperations().getProperties(t);
+      Assert.fail();
+    } catch (TableNotFoundException e) {}
+    try {
+      conn.tableOperations().listSplits(t);
+      Assert.fail();
+    } catch (TableNotFoundException e) {}
+    try {
+      conn.tableOperations().listIterators(t);
+      Assert.fail();
+    } catch (TableNotFoundException e) {}
+    try {
+      conn.tableOperations().removeIterator(t, null, 
EnumSet.noneOf(IteratorScope.class));
+      Assert.fail();
+    } catch (TableNotFoundException e) {}
+    try {
+      conn.tableOperations().rename(t, t);
+      Assert.fail();
+    } catch (TableNotFoundException e) {}
+    conn.tableOperations().create(t);
+    try {
+      conn.tableOperations().create(t);
+      Assert.fail();
+    } catch (TableExistsException e) {}
+    try {
+      conn.tableOperations().rename(t, t);
+      Assert.fail();
+    } catch (TableExistsException e) {}
+  }
+
+  private static class ImportTestFilesAndData {
+    Path importPath;
+    Path failurePath;
+    List<Pair<Key,Value>> keyVals;
+  }
+
+  @Test
+  public void testImport() throws Throwable {
+    ImportTestFilesAndData dataAndFiles = prepareTestFiles();
+    TableOperations tableOperations = conn.tableOperations();
+    tableOperations.create("a_table");
+    tableOperations.importDirectory("a_table", 
dataAndFiles.importPath.toString(), dataAndFiles.failurePath.toString(), false);
+    Scanner scanner = conn.createScanner("a_table", new Authorizations());
+    Iterator<Entry<Key,Value>> iterator = scanner.iterator();
+    for (int i = 0; i < 5; i++) {
+      Assert.assertTrue(iterator.hasNext());
+      Entry<Key,Value> kv = iterator.next();
+      Pair<Key,Value> expected = dataAndFiles.keyVals.get(i);
+      Assert.assertEquals(expected.getFirst(), kv.getKey());
+      Assert.assertEquals(expected.getSecond(), kv.getValue());
+    }
+    Assert.assertFalse(iterator.hasNext());
+  }
+
+  private ImportTestFilesAndData prepareTestFiles() throws Throwable {
+    Configuration defaultConf = new Configuration();
+    Path tempFile = new Path("target/accumulo-test/import/sample.rf");
+    Path failures = new Path("target/accumulo-test/failures/");
+    FileSystem fs = FileSystem.get(new URI("file:///"), defaultConf);
+    fs.deleteOnExit(tempFile);
+    fs.deleteOnExit(failures);
+    fs.delete(failures, true);
+    fs.delete(tempFile, true);
+    fs.mkdirs(failures);
+    fs.mkdirs(tempFile.getParent());
+    FileSKVWriter writer = 
FileOperations.getInstance().newWriterBuilder().forFile(tempFile.toString(), 
fs, defaultConf)
+        
.withTableConfiguration(AccumuloConfiguration.getDefaultConfiguration()).build();
+    writer.startDefaultLocalityGroup();
+    List<Pair<Key,Value>> keyVals = new ArrayList<>();
+    for (int i = 0; i < 5; i++) {
+      keyVals.add(new Pair<>(new Key("a" + i, "b" + i, "c" + i, new 
ColumnVisibility(""), 1000l + i), new Value(Integer.toString(i).getBytes())));
+    }
+    for (Pair<Key,Value> keyVal : keyVals) {
+      writer.append(keyVal.getFirst(), keyVal.getSecond());
+    }
+    writer.close();
+    ImportTestFilesAndData files = new ImportTestFilesAndData();
+    files.failurePath = failures;
+    files.importPath = tempFile.getParent();
+    files.keyVals = keyVals;
+    return files;
+  }
+
+  @Test(expected = TableNotFoundException.class)
+  public void testFailsWithNoTable() throws Throwable {
+    TableOperations tableOperations = conn.tableOperations();
+    ImportTestFilesAndData testFiles = prepareTestFiles();
+    tableOperations.importDirectory("doesnt_exist_table", 
testFiles.importPath.toString(), testFiles.failurePath.toString(), false);
+  }
+
+  @Test(expected = IOException.class)
+  public void testFailsWithNonEmptyFailureDirectory() throws Throwable {
+    TableOperations tableOperations = conn.tableOperations();
+    ImportTestFilesAndData testFiles = prepareTestFiles();
+    FileSystem fs = testFiles.failurePath.getFileSystem(new Configuration());
+    fs.open(testFiles.failurePath.suffix("/something")).close();
+    tableOperations.importDirectory("doesnt_exist_table", 
testFiles.importPath.toString(), testFiles.failurePath.toString(), false);
+  }
+
+  @Test
+  public void testDeleteRows() throws Exception {
+    TableOperations to = conn.tableOperations();
+    to.create("test");
+    BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig());
+    for (int r = 0; r < 20; r++) {
+      Mutation m = new Mutation("" + r);
+      for (int c = 0; c < 5; c++) {
+        m.put(new Text("cf"), new Text("" + c), new Value(("" + 
c).getBytes()));
+      }
+      bw.addMutation(m);
+    }
+    bw.flush();
+    to.deleteRows("test", new Text("1"), new Text("2"));
+    Scanner s = conn.createScanner("test", Authorizations.EMPTY);
+    int oneCnt = 0;
+    for (Entry<Key,Value> entry : s) {
+      char rowStart = entry.getKey().getRow().toString().charAt(0);
+      Assert.assertTrue(rowStart != '2');
+      oneCnt += rowStart == '1' ? 1 : 0;
+    }
+    Assert.assertEquals(5, oneCnt);
+  }
+
+  @Test
+  public void testDeleteRowsWithNullKeys() throws Exception {
+    TableOperations to = conn.tableOperations();
+    to.create("test2");
+    BatchWriter bw = conn.createBatchWriter("test2", new BatchWriterConfig());
+    for (int r = 0; r < 30; r++) {
+      Mutation m = new Mutation(Integer.toString(r));
+      for (int c = 0; c < 5; c++) {
+        m.put(new Text("cf"), new Text(Integer.toString(c)), new 
Value(Integer.toString(c).getBytes()));
+      }
+      bw.addMutation(m);
+    }
+    bw.flush();
+
+    // test null end
+    // will remove rows 4 through 9 (6 * 5 = 30 entries)
+    to.deleteRows("test2", new Text("30"), null);
+    Scanner s = conn.createScanner("test2", Authorizations.EMPTY);
+    int rowCnt = 0;
+    for (Entry<Key,Value> entry : s) {
+      String rowId = entry.getKey().getRow().toString();
+      Assert.assertFalse(rowId.startsWith("30"));
+      rowCnt++;
+    }
+    s.close();
+    Assert.assertEquals(120, rowCnt);
+
+    // test null start
+    // will remove 0-1, 10-19, 2
+    to.deleteRows("test2", null, new Text("2"));
+    s = conn.createScanner("test2", Authorizations.EMPTY);
+    rowCnt = 0;
+    for (Entry<Key,Value> entry : s) {
+      char rowStart = entry.getKey().getRow().toString().charAt(0);
+      Assert.assertTrue(rowStart >= '2');
+      rowCnt++;
+    }
+    s.close();
+    Assert.assertEquals(55, rowCnt);
+
+    // test null start and end
+    // deletes everything still left
+    to.deleteRows("test2", null, null);
+    s = conn.createScanner("test2", Authorizations.EMPTY);
+    rowCnt = Iterators.size(s.iterator());
+    s.close();
+    to.delete("test2");
+    Assert.assertEquals(0, rowCnt);
+
+  }
+
+  @Test
+  public void testTableIdMap() throws Exception {
+    TableOperations tops = conn.tableOperations();
+    tops.create("foo");
+
+    // Should get a table ID, not the table name
+    Assert.assertNotEquals("foo", tops.tableIdMap().get("foo"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/core/src/test/java/org/apache/accumulo/core/client/mock/TestBatchScanner821.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/accumulo/core/client/mock/TestBatchScanner821.java
 
b/core/src/test/java/org/apache/accumulo/core/client/mock/TestBatchScanner821.java
new file mode 100644
index 0000000..4f041c9
--- /dev/null
+++ 
b/core/src/test/java/org/apache/accumulo/core/client/mock/TestBatchScanner821.java
@@ -0,0 +1,77 @@
+/*
+ * 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.mock;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Collections;
+import java.util.Map.Entry;
+
+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.Connector;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.WrappingIterator;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+@Deprecated
+public class TestBatchScanner821 {
+
+  public static class TransformIterator extends WrappingIterator {
+
+    @Override
+    public Key getTopKey() {
+      Key k = getSource().getTopKey();
+      return new Key(new Text(k.getRow().toString().toLowerCase()), 
k.getColumnFamily(), k.getColumnQualifier(), k.getColumnVisibility(), 
k.getTimestamp());
+    }
+  }
+
+  @Test
+  public void test() throws Exception {
+    MockInstance inst = new MockInstance();
+    Connector conn = inst.getConnector("root", new PasswordToken(""));
+    conn.tableOperations().create("test");
+    BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig());
+    for (String row : "A,B,C,D".split(",")) {
+      Mutation m = new Mutation(row);
+      m.put("cf", "cq", "");
+      bw.addMutation(m);
+    }
+    bw.flush();
+    BatchScanner bs = conn.createBatchScanner("test", Authorizations.EMPTY, 1);
+    IteratorSetting cfg = new IteratorSetting(100, TransformIterator.class);
+    bs.addScanIterator(cfg);
+    bs.setRanges(Collections.singletonList(new Range("A", "Z")));
+    StringBuilder sb = new StringBuilder();
+    String comma = "";
+    for (Entry<Key,Value> entry : bs) {
+      sb.append(comma);
+      sb.append(entry.getKey().getRow());
+      comma = ",";
+    }
+    assertEquals("a,b,c,d", sb.toString());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/core/src/test/java/org/apache/accumulo/core/data/KeyExtentTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/accumulo/core/data/KeyExtentTest.java 
b/core/src/test/java/org/apache/accumulo/core/data/KeyExtentTest.java
index 0bff486..79968be 100644
--- a/core/src/test/java/org/apache/accumulo/core/data/KeyExtentTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/data/KeyExtentTest.java
@@ -16,9 +16,13 @@
  */
 package org.apache.accumulo.core.data;
 
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.CoreMatchers.hasItems;
+import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayInputStream;
@@ -26,6 +30,9 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.SortedMap;
 import java.util.TreeMap;
 import java.util.TreeSet;
@@ -285,6 +292,57 @@ public class KeyExtentTest {
     return out;
   }
 
+  @SuppressWarnings("deprecation")
+  @Test
+  public void testKeyExtentsForSimpleRange() {
+    Collection<KeyExtent> results;
+
+    results = KeyExtent.getKeyExtentsForRange(null, null, null);
+    assertTrue("Non-empty set returned from no extents", results.isEmpty());
+
+    results = KeyExtent.getKeyExtentsForRange(null, null, 
Collections.<KeyExtent> emptySet());
+    assertTrue("Non-empty set returned from no extents", results.isEmpty());
+
+    KeyExtent t = nke("t", null, null);
+    results = KeyExtent.getKeyExtentsForRange(null, null, 
Collections.<KeyExtent> singleton(t));
+    assertEquals("Single tablet should always be returned", 1, results.size());
+    assertEquals(t, results.iterator().next());
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test
+  public void testKeyExtentsForRange() {
+    KeyExtent b = nke("t", "b", null);
+    KeyExtent e = nke("t", "e", "b");
+    KeyExtent h = nke("t", "h", "e");
+    KeyExtent m = nke("t", "m", "h");
+    KeyExtent z = nke("t", null, "m");
+
+    set0.addAll(Arrays.asList(b, e, h, m, z));
+
+    Collection<KeyExtent> results;
+
+    results = KeyExtent.getKeyExtentsForRange(null, null, set0);
+    assertThat("infinite range should return full set", results.size(), is(5));
+    assertThat("infinite range should return full set", results, hasItems(b, 
e, h, m, z));
+
+    results = KeyExtent.getKeyExtentsForRange(new Text("a"), new Text("z"), 
set0);
+    assertThat("full overlap should return full set", results.size(), is(5));
+    assertThat("full overlap should return full set", results, hasItems(b, e, 
h, m, z));
+
+    results = KeyExtent.getKeyExtentsForRange(null, new Text("f"), set0);
+    assertThat("end row should return head set", results.size(), is(3));
+    assertThat("end row should return head set", results, hasItems(b, e, h));
+
+    results = KeyExtent.getKeyExtentsForRange(new Text("f"), null, set0);
+    assertThat("start row should return tail set", results.size(), is(3));
+    assertThat("start row should return tail set", results, hasItems(h, m, z));
+
+    results = KeyExtent.getKeyExtentsForRange(new Text("f"), new Text("g"), 
set0);
+    assertThat("slice should return correct subset", results.size(), is(1));
+    assertThat("slice should return correct subset", results, hasItem(h));
+  }
+
   @Test
   public void testDecodeEncode() {
     assertNull(KeyExtent.decodePrevEndRow(KeyExtent.encodePrevEndRow(null)));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/core/src/test/java/org/apache/accumulo/core/data/ValueTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/data/ValueTest.java 
b/core/src/test/java/org/apache/accumulo/core/data/ValueTest.java
index 0c0042b..93fab1f 100644
--- a/core/src/test/java/org/apache/accumulo/core/data/ValueTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/data/ValueTest.java
@@ -33,6 +33,7 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.nio.ByteBuffer;
+import java.util.List;
 
 import org.apache.hadoop.io.Text;
 import org.junit.Before;
@@ -102,6 +103,13 @@ public class ValueTest {
   }
 
   @Test
+  public void testByteBufferCopy() {
+    @SuppressWarnings("deprecation")
+    Value v = new Value(DATABUFF, true);
+    assertArrayEquals(DATA, v.get());
+  }
+
+  @Test
   public void testValueCopy() {
     Value ov = createMock(Value.class);
     expect(ov.get()).andReturn(DATA);
@@ -192,6 +200,24 @@ public class ValueTest {
   }
 
   @Test
+  @Deprecated
+  public void testToArray() {
+    List<byte[]> l = new java.util.ArrayList<>();
+    byte[] one = toBytes("one");
+    byte[] two = toBytes("two");
+    byte[] three = toBytes("three");
+    l.add(one);
+    l.add(two);
+    l.add(three);
+
+    byte[][] a = Value.toArray(l);
+    assertEquals(3, a.length);
+    assertArrayEquals(one, a[0]);
+    assertArrayEquals(two, a[1]);
+    assertArrayEquals(three, a[2]);
+  }
+
+  @Test
   public void testString() {
     Value v1 = new Value("abc");
     Value v2 = new Value("abc".getBytes(UTF_8));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/core/src/test/java/org/apache/accumulo/core/iterators/AggregatingIteratorTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/accumulo/core/iterators/AggregatingIteratorTest.java
 
b/core/src/test/java/org/apache/accumulo/core/iterators/AggregatingIteratorTest.java
new file mode 100644
index 0000000..09064a5
--- /dev/null
+++ 
b/core/src/test/java/org/apache/accumulo/core/iterators/AggregatingIteratorTest.java
@@ -0,0 +1,471 @@
+/*
+ * 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.iterators;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.system.MultiIterator;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class AggregatingIteratorTest {
+
+  private static final Collection<ByteSequence> EMPTY_COL_FAMS = new 
ArrayList<>();
+
+  /**
+   * @deprecated since 1.4; visible only for testing
+   */
+  @Deprecated
+  public static class SummationAggregator implements 
org.apache.accumulo.core.iterators.aggregation.Aggregator {
+
+    int sum;
+
+    @Override
+    public Value aggregate() {
+      return new Value((sum + "").getBytes());
+    }
+
+    @Override
+    public void collect(Value value) {
+      int val = Integer.parseInt(value.toString());
+
+      sum += val;
+    }
+
+    @Override
+    public void reset() {
+      sum = 0;
+
+    }
+
+  }
+
+  static Key nk(int row, int colf, int colq, long ts, boolean deleted) {
+    Key k = nk(row, colf, colq, ts);
+    k.setDeleted(true);
+    return k;
+  }
+
+  static Key nk(int row, int colf, int colq, long ts) {
+    return new Key(nr(row), new Text(String.format("cf%03d", colf)), new 
Text(String.format("cq%03d", colq)), ts);
+  }
+
+  static Range nr(int row, int colf, int colq, long ts, boolean inclusive) {
+    return new Range(nk(row, colf, colq, ts), inclusive, null, true);
+  }
+
+  static Range nr(int row, int colf, int colq, long ts) {
+    return nr(row, colf, colq, ts, true);
+  }
+
+  static void nkv(TreeMap<Key,Value> tm, int row, int colf, int colq, long ts, 
boolean deleted, String val) {
+    Key k = nk(row, colf, colq, ts);
+    k.setDeleted(deleted);
+    tm.put(k, new Value(val.getBytes()));
+  }
+
+  static Text nr(int row) {
+    return new Text(String.format("r%03d", row));
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test
+  public void test1() throws IOException {
+
+    TreeMap<Key,Value> tm1 = new TreeMap<>();
+
+    // keys that do not aggregate
+    nkv(tm1, 1, 1, 1, 1, false, "2");
+    nkv(tm1, 1, 1, 1, 2, false, "3");
+    nkv(tm1, 1, 1, 1, 3, false, "4");
+
+    AggregatingIterator ai = new AggregatingIterator();
+
+    Map<String,String> emptyMap = Collections.emptyMap();
+    ai.init(new SortedMapIterator(tm1), emptyMap, null);
+    ai.seek(new Range(), EMPTY_COL_FAMS, false);
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(1, 1, 1, 3), ai.getTopKey());
+    assertEquals("4", ai.getTopValue().toString());
+
+    ai.next();
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(1, 1, 1, 2), ai.getTopKey());
+    assertEquals("3", ai.getTopValue().toString());
+
+    ai.next();
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(1, 1, 1, 1), ai.getTopKey());
+    assertEquals("2", ai.getTopValue().toString());
+
+    ai.next();
+
+    assertFalse(ai.hasTop());
+
+    // try seeking
+
+    ai.seek(nr(1, 1, 1, 2), EMPTY_COL_FAMS, false);
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(1, 1, 1, 2), ai.getTopKey());
+    assertEquals("3", ai.getTopValue().toString());
+
+    ai.next();
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(1, 1, 1, 1), ai.getTopKey());
+    assertEquals("2", ai.getTopValue().toString());
+
+    ai.next();
+
+    assertFalse(ai.hasTop());
+
+    // seek after everything
+    ai.seek(nr(1, 1, 1, 0), EMPTY_COL_FAMS, false);
+
+    assertFalse(ai.hasTop());
+
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test
+  public void test2() throws IOException {
+    TreeMap<Key,Value> tm1 = new TreeMap<>();
+
+    // keys that aggregate
+    nkv(tm1, 1, 1, 1, 1, false, "2");
+    nkv(tm1, 1, 1, 1, 2, false, "3");
+    nkv(tm1, 1, 1, 1, 3, false, "4");
+
+    AggregatingIterator ai = new AggregatingIterator();
+
+    Map<String,String> opts = new HashMap<>();
+
+    opts.put("cf001", SummationAggregator.class.getName());
+
+    ai.init(new SortedMapIterator(tm1), opts, null);
+    ai.seek(new Range(), EMPTY_COL_FAMS, false);
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(1, 1, 1, 3), ai.getTopKey());
+    assertEquals("9", ai.getTopValue().toString());
+
+    ai.next();
+
+    assertFalse(ai.hasTop());
+
+    // try seeking to the beginning of a key that aggregates
+
+    ai.seek(nr(1, 1, 1, 3), EMPTY_COL_FAMS, false);
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(1, 1, 1, 3), ai.getTopKey());
+    assertEquals("9", ai.getTopValue().toString());
+
+    ai.next();
+
+    assertFalse(ai.hasTop());
+
+    // try seeking the middle of a key the aggregates
+    ai.seek(nr(1, 1, 1, 2), EMPTY_COL_FAMS, false);
+
+    assertFalse(ai.hasTop());
+
+    // try seeking to the end of a key the aggregates
+    ai.seek(nr(1, 1, 1, 1), EMPTY_COL_FAMS, false);
+
+    assertFalse(ai.hasTop());
+
+    // try seeking before a key the aggregates
+    ai.seek(nr(1, 1, 1, 4), EMPTY_COL_FAMS, false);
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(1, 1, 1, 3), ai.getTopKey());
+    assertEquals("9", ai.getTopValue().toString());
+
+    ai.next();
+
+    assertFalse(ai.hasTop());
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test
+  public void test3() throws IOException {
+
+    TreeMap<Key,Value> tm1 = new TreeMap<>();
+
+    // keys that aggregate
+    nkv(tm1, 1, 1, 1, 1, false, "2");
+    nkv(tm1, 1, 1, 1, 2, false, "3");
+    nkv(tm1, 1, 1, 1, 3, false, "4");
+
+    // keys that do not aggregate
+    nkv(tm1, 2, 2, 1, 1, false, "2");
+    nkv(tm1, 2, 2, 1, 2, false, "3");
+
+    AggregatingIterator ai = new AggregatingIterator();
+
+    Map<String,String> opts = new HashMap<>();
+
+    opts.put("cf001", SummationAggregator.class.getName());
+
+    ai.init(new SortedMapIterator(tm1), opts, null);
+    ai.seek(new Range(), EMPTY_COL_FAMS, false);
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(1, 1, 1, 3), ai.getTopKey());
+    assertEquals("9", ai.getTopValue().toString());
+
+    ai.next();
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(2, 2, 1, 2), ai.getTopKey());
+    assertEquals("3", ai.getTopValue().toString());
+
+    ai.next();
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(2, 2, 1, 1), ai.getTopKey());
+    assertEquals("2", ai.getTopValue().toString());
+
+    ai.next();
+
+    assertFalse(ai.hasTop());
+
+    // seek after key that aggregates
+    ai.seek(nr(1, 1, 1, 2), EMPTY_COL_FAMS, false);
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(2, 2, 1, 2), ai.getTopKey());
+    assertEquals("3", ai.getTopValue().toString());
+
+    // seek before key that aggregates
+    ai.seek(nr(1, 1, 1, 4), EMPTY_COL_FAMS, false);
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(1, 1, 1, 3), ai.getTopKey());
+    assertEquals("9", ai.getTopValue().toString());
+
+    ai.next();
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(2, 2, 1, 2), ai.getTopKey());
+    assertEquals("3", ai.getTopValue().toString());
+
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test
+  public void test4() throws IOException {
+
+    TreeMap<Key,Value> tm1 = new TreeMap<>();
+
+    // keys that do not aggregate
+    nkv(tm1, 0, 0, 1, 1, false, "7");
+
+    // keys that aggregate
+    nkv(tm1, 1, 1, 1, 1, false, "2");
+    nkv(tm1, 1, 1, 1, 2, false, "3");
+    nkv(tm1, 1, 1, 1, 3, false, "4");
+
+    // keys that do not aggregate
+    nkv(tm1, 2, 2, 1, 1, false, "2");
+    nkv(tm1, 2, 2, 1, 2, false, "3");
+
+    AggregatingIterator ai = new AggregatingIterator();
+
+    Map<String,String> opts = new HashMap<>();
+
+    opts.put("cf001", SummationAggregator.class.getName());
+
+    ai.init(new SortedMapIterator(tm1), opts, null);
+    ai.seek(new Range(), EMPTY_COL_FAMS, false);
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(0, 0, 1, 1), ai.getTopKey());
+    assertEquals("7", ai.getTopValue().toString());
+
+    ai.next();
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(1, 1, 1, 3), ai.getTopKey());
+    assertEquals("9", ai.getTopValue().toString());
+
+    ai.next();
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(2, 2, 1, 2), ai.getTopKey());
+    assertEquals("3", ai.getTopValue().toString());
+
+    ai.next();
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(2, 2, 1, 1), ai.getTopKey());
+    assertEquals("2", ai.getTopValue().toString());
+
+    ai.next();
+
+    assertFalse(ai.hasTop());
+
+    // seek test
+    ai.seek(nr(0, 0, 1, 0), EMPTY_COL_FAMS, false);
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(1, 1, 1, 3), ai.getTopKey());
+    assertEquals("9", ai.getTopValue().toString());
+
+    ai.next();
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(2, 2, 1, 2), ai.getTopKey());
+    assertEquals("3", ai.getTopValue().toString());
+
+    // seek after key that aggregates
+    ai.seek(nr(1, 1, 1, 2), EMPTY_COL_FAMS, false);
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(2, 2, 1, 2), ai.getTopKey());
+    assertEquals("3", ai.getTopValue().toString());
+
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test
+  public void test5() throws IOException {
+    // try aggregating across multiple data sets that contain
+    // the exact same keys w/ different values
+
+    TreeMap<Key,Value> tm1 = new TreeMap<>();
+    nkv(tm1, 1, 1, 1, 1, false, "2");
+
+    TreeMap<Key,Value> tm2 = new TreeMap<>();
+    nkv(tm2, 1, 1, 1, 1, false, "3");
+
+    TreeMap<Key,Value> tm3 = new TreeMap<>();
+    nkv(tm3, 1, 1, 1, 1, false, "4");
+
+    AggregatingIterator ai = new AggregatingIterator();
+    Map<String,String> opts = new HashMap<>();
+    opts.put("cf001", SummationAggregator.class.getName());
+
+    List<SortedKeyValueIterator<Key,Value>> sources = new ArrayList<>(3);
+    sources.add(new SortedMapIterator(tm1));
+    sources.add(new SortedMapIterator(tm2));
+    sources.add(new SortedMapIterator(tm3));
+
+    MultiIterator mi = new MultiIterator(sources, true);
+    ai.init(mi, opts, null);
+    ai.seek(new Range(), EMPTY_COL_FAMS, false);
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(1, 1, 1, 1), ai.getTopKey());
+    assertEquals("9", ai.getTopValue().toString());
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test
+  public void test6() throws IOException {
+    TreeMap<Key,Value> tm1 = new TreeMap<>();
+
+    // keys that aggregate
+    nkv(tm1, 1, 1, 1, 1, false, "2");
+    nkv(tm1, 1, 1, 1, 2, false, "3");
+    nkv(tm1, 1, 1, 1, 3, false, "4");
+
+    AggregatingIterator ai = new AggregatingIterator();
+
+    Map<String,String> opts = new HashMap<>();
+
+    opts.put("cf001", SummationAggregator.class.getName());
+
+    ai.init(new SortedMapIterator(tm1), opts, new 
DefaultIteratorEnvironment());
+
+    // try seeking to the beginning of a key that aggregates
+
+    ai.seek(nr(1, 1, 1, 3, false), EMPTY_COL_FAMS, false);
+
+    assertFalse(ai.hasTop());
+
+  }
+
+  @SuppressWarnings("deprecation")
+  @Test
+  public void test7() throws IOException {
+    // test that delete is not aggregated
+
+    TreeMap<Key,Value> tm1 = new TreeMap<>();
+
+    nkv(tm1, 1, 1, 1, 2, true, "");
+    nkv(tm1, 1, 1, 1, 3, false, "4");
+    nkv(tm1, 1, 1, 1, 4, false, "3");
+
+    AggregatingIterator ai = new AggregatingIterator();
+
+    Map<String,String> opts = new HashMap<>();
+
+    opts.put("cf001", SummationAggregator.class.getName());
+
+    ai.init(new SortedMapIterator(tm1), opts, new 
DefaultIteratorEnvironment());
+
+    ai.seek(nr(1, 1, 1, 4, true), EMPTY_COL_FAMS, false);
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(1, 1, 1, 4), ai.getTopKey());
+    assertEquals("7", ai.getTopValue().toString());
+
+    ai.next();
+    assertTrue(ai.hasTop());
+    assertEquals(nk(1, 1, 1, 2, true), ai.getTopKey());
+    assertEquals("", ai.getTopValue().toString());
+
+    ai.next();
+    assertFalse(ai.hasTop());
+
+    tm1 = new TreeMap<>();
+    nkv(tm1, 1, 1, 1, 2, true, "");
+    ai = new AggregatingIterator();
+    ai.init(new SortedMapIterator(tm1), opts, new 
DefaultIteratorEnvironment());
+
+    ai.seek(nr(1, 1, 1, 4, true), EMPTY_COL_FAMS, false);
+
+    assertTrue(ai.hasTop());
+    assertEquals(nk(1, 1, 1, 2, true), ai.getTopKey());
+    assertEquals("", ai.getTopValue().toString());
+
+    ai.next();
+    assertFalse(ai.hasTop());
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/core/src/test/java/org/apache/accumulo/core/iterators/aggregation/NumSummationTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/accumulo/core/iterators/aggregation/NumSummationTest.java
 
b/core/src/test/java/org/apache/accumulo/core/iterators/aggregation/NumSummationTest.java
new file mode 100644
index 0000000..5a56ead
--- /dev/null
+++ 
b/core/src/test/java/org/apache/accumulo/core/iterators/aggregation/NumSummationTest.java
@@ -0,0 +1,149 @@
+/*
+ * 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.iterators.aggregation;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
+import org.apache.accumulo.core.data.Value;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * @deprecated since 1.4
+ */
+@Deprecated
+public class NumSummationTest {
+
+  private static final Logger log = 
LoggerFactory.getLogger(NumSummationTest.class);
+
+  public byte[] init(int n) {
+    byte[] b = new byte[n];
+    for (int i = 0; i < b.length; i++)
+      b[i] = 0;
+    return b;
+  }
+
+  @Test
+  public void test1() {
+    try {
+      long[] la = {1l, 2l, 3l};
+      byte[] b = NumArraySummation.longArrayToBytes(la);
+      long[] la2 = NumArraySummation.bytesToLongArray(b);
+
+      assertTrue(la.length == la2.length);
+      for (int i = 0; i < la.length; i++) {
+        assertTrue(i + ": " + la[i] + " does not equal " + la2[i], la[i] == 
la2[i]);
+      }
+    } catch (Exception e) {
+      assertTrue(false);
+    }
+  }
+
+  @Test
+  public void test2() {
+    try {
+      NumArraySummation nas = new NumArraySummation();
+      long[] la = {1l, 2l, 3l};
+      nas.collect(new Value(NumArraySummation.longArrayToBytes(la)));
+      long[] la2 = {3l, 2l, 1l, 0l};
+      nas.collect(new Value(NumArraySummation.longArrayToBytes(la2)));
+      la = NumArraySummation.bytesToLongArray(nas.aggregate().get());
+      assertTrue(la.length == 4);
+      for (int i = 0; i < la.length - 1; i++) {
+        assertTrue(la[i] == 4);
+      }
+      assertTrue(la[la.length - 1] == 0);
+      nas.reset();
+      la = NumArraySummation.bytesToLongArray(nas.aggregate().get());
+      assertTrue(la.length == 0);
+    } catch (Exception e) {
+      log.error("{}", e.getMessage(), e);
+      assertTrue(false);
+    }
+  }
+
+  @Test
+  public void test3() {
+    try {
+      NumArraySummation nas = new NumArraySummation();
+      long[] la = {Long.MAX_VALUE, Long.MIN_VALUE, 3l, -5l, 5l, 5l};
+      nas.collect(new Value(NumArraySummation.longArrayToBytes(la)));
+      long[] la2 = {1l, -3l, 2l, 10l};
+      nas.collect(new Value(NumArraySummation.longArrayToBytes(la2)));
+      la = NumArraySummation.bytesToLongArray(nas.aggregate().get());
+      assertTrue(la.length == 6);
+      for (int i = 2; i < la.length; i++) {
+        assertTrue(la[i] == 5);
+      }
+      assertTrue("max long plus one was " + la[0], la[0] == Long.MAX_VALUE);
+      assertTrue("min long minus 3 was " + la[1], la[1] == Long.MIN_VALUE);
+    } catch (Exception e) {
+      assertTrue(false);
+    }
+  }
+
+  @Test
+  public void test4() {
+    try {
+      long l = 5l;
+      byte[] b = NumSummation.longToBytes(l);
+      long l2 = NumSummation.bytesToLong(b);
+
+      assertTrue(l == l2);
+    } catch (Exception e) {
+      assertTrue(false);
+    }
+  }
+
+  @Test
+  public void test5() {
+    try {
+      NumSummation ns = new NumSummation();
+      for (long l = -5l; l < 8l; l++) {
+        ns.collect(new Value(NumSummation.longToBytes(l)));
+      }
+      long l = NumSummation.bytesToLong(ns.aggregate().get());
+      assertTrue("l was " + l, l == 13);
+
+      ns.collect(new Value(NumSummation.longToBytes(Long.MAX_VALUE)));
+      l = NumSummation.bytesToLong(ns.aggregate().get());
+      assertTrue("l was " + l, l == Long.MAX_VALUE);
+
+      ns.collect(new Value(NumSummation.longToBytes(Long.MIN_VALUE)));
+      l = NumSummation.bytesToLong(ns.aggregate().get());
+      assertTrue("l was " + l, l == -1);
+
+      ns.collect(new Value(NumSummation.longToBytes(Long.MIN_VALUE)));
+      l = NumSummation.bytesToLong(ns.aggregate().get());
+      assertTrue("l was " + l, l == Long.MIN_VALUE);
+
+      ns.collect(new Value(NumSummation.longToBytes(Long.MIN_VALUE)));
+      l = NumSummation.bytesToLong(ns.aggregate().get());
+      assertTrue("l was " + l, l == Long.MIN_VALUE);
+
+      ns.reset();
+      l = NumSummation.bytesToLong(ns.aggregate().get());
+      assertTrue("l was " + l, l == 0);
+    } catch (IOException | RuntimeException e) {
+      fail();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/core/src/test/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorConfigurationTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorConfigurationTest.java
 
b/core/src/test/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorConfigurationTest.java
new file mode 100644
index 0000000..61693ab
--- /dev/null
+++ 
b/core/src/test/java/org/apache/accumulo/core/iterators/aggregation/conf/AggregatorConfigurationTest.java
@@ -0,0 +1,72 @@
+/*
+ * 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.iterators.aggregation.conf;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class AggregatorConfigurationTest {
+
+  @Test
+  public void testBinary() {
+    Text colf = new Text();
+    Text colq = new Text();
+
+    for (int i = 0; i < 256; i++) {
+      colf.append(new byte[] {(byte) i}, 0, 1);
+      colq.append(new byte[] {(byte) (255 - i)}, 0, 1);
+    }
+
+    runTest(colf, colq);
+    runTest(colf);
+  }
+
+  @Test
+  public void testBasic() {
+    runTest(new Text("colf1"), new Text("cq2"));
+    runTest(new Text("colf1"));
+  }
+
+  @SuppressWarnings("deprecation")
+  private void runTest(Text colf) {
+    String encodedCols;
+    org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig ac3 = new 
org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig(colf,
+        "com.foo.SuperAgg");
+    encodedCols = ac3.encodeColumns();
+    org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig ac4 = 
org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig.decodeColumns(
+        encodedCols, "com.foo.SuperAgg");
+
+    assertEquals(colf, ac4.getColumnFamily());
+    assertNull(ac4.getColumnQualifier());
+  }
+
+  @SuppressWarnings("deprecation")
+  private void runTest(Text colf, Text colq) {
+    org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig ac = new 
org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig(colf, colq,
+        "com.foo.SuperAgg");
+    String encodedCols = ac.encodeColumns();
+    org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig ac2 = 
org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig.decodeColumns(
+        encodedCols, "com.foo.SuperAgg");
+
+    assertEquals(colf, ac2.getColumnFamily());
+    assertEquals(colq, ac2.getColumnQualifier());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/core/src/test/java/org/apache/accumulo/core/iterators/user/FilterTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/accumulo/core/iterators/user/FilterTest.java 
b/core/src/test/java/org/apache/accumulo/core/iterators/user/FilterTest.java
index 6546352..b26c218 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/FilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/FilterTest.java
@@ -258,7 +258,7 @@ public class FilterTest {
     Text colf = new Text("a");
     Text colq = new Text("b");
     Value dv = new Value();
-    TreeMap<Key,Value> tm = new TreeMap<Key,Value>();
+    TreeMap<Key,Value> tm = new TreeMap<>();
     IteratorSetting is = new IteratorSetting(1, ColumnAgeOffFilter.class);
     ColumnAgeOffFilter.addTTL(is, new IteratorSetting.Column("a"), 901l);
     ColumnAgeOffFilter.setNegate(is, true);
@@ -299,7 +299,7 @@ public class FilterTest {
     Text colf = new Text("negate");
     Text colq = new Text("b");
     Value dv = new Value();
-    TreeMap<Key,Value> tm = new TreeMap<Key,Value>();
+    TreeMap<Key,Value> tm = new TreeMap<>();
     IteratorSetting is = new IteratorSetting(1, ColumnAgeOffFilter.class);
     ColumnAgeOffFilter.addTTL(is, new IteratorSetting.Column("negate"), 901l);
     long ts = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/core/src/test/java/org/apache/accumulo/core/security/CredentialsTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/accumulo/core/security/CredentialsTest.java 
b/core/src/test/java/org/apache/accumulo/core/security/CredentialsTest.java
index 8422c6f..bd4b1ba 100644
--- a/core/src/test/java/org/apache/accumulo/core/security/CredentialsTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/security/CredentialsTest.java
@@ -25,7 +25,9 @@ import static org.junit.Assert.fail;
 
 import javax.security.auth.DestroyFailedException;
 
+import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
@@ -33,6 +35,7 @@ import 
org.apache.accumulo.core.client.security.tokens.AuthenticationToken.Authe
 import org.apache.accumulo.core.client.security.tokens.NullToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.easymock.EasyMock;
 import org.junit.Before;
 import org.junit.Rule;
@@ -83,6 +86,24 @@ public class CredentialsTest {
   }
 
   @Test
+  public void testMockConnector() throws AccumuloException, 
DestroyFailedException, AccumuloSecurityException {
+    Instance inst = DeprecationUtil.makeMockInstance(test.getMethodName());
+    Connector rootConnector = inst.getConnector("root", new PasswordToken());
+    PasswordToken testToken = new PasswordToken("testPass");
+    rootConnector.securityOperations().createLocalUser("testUser", testToken);
+
+    assertFalse(testToken.isDestroyed());
+    testToken.destroy();
+    assertTrue(testToken.isDestroyed());
+    try {
+      inst.getConnector("testUser", testToken);
+      fail();
+    } catch (AccumuloSecurityException e) {
+      
assertTrue(e.getSecurityErrorCode().equals(SecurityErrorCode.TOKEN_EXPIRED));
+    }
+  }
+
+  @Test
   public void testEqualsAndHashCode() {
     Credentials nullNullCreds = new Credentials(null, null);
     Credentials abcNullCreds = new Credentials("abc", new NullToken());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/core/src/test/java/org/apache/accumulo/core/util/format/DateStringFormatterTest.java
----------------------------------------------------------------------
diff --git 
a/core/src/test/java/org/apache/accumulo/core/util/format/DateStringFormatterTest.java
 
b/core/src/test/java/org/apache/accumulo/core/util/format/DateStringFormatterTest.java
new file mode 100644
index 0000000..22af5b0
--- /dev/null
+++ 
b/core/src/test/java/org/apache/accumulo/core/util/format/DateStringFormatterTest.java
@@ -0,0 +1,80 @@
+/*
+ * 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.util.format;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Map;
+import java.util.TimeZone;
+import java.util.TreeMap;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.junit.Before;
+import org.junit.Test;
+
+@SuppressWarnings("deprecation")
+public class DateStringFormatterTest {
+  DateStringFormatter formatter;
+
+  Map<Key,Value> data;
+
+  @Before
+  public void setUp() {
+    formatter = new DateStringFormatter();
+    data = new TreeMap<>();
+    data.put(new Key("", "", "", 0), new Value());
+  }
+
+  private void testFormatterIgnoresConfig(FormatterConfig config, 
DateStringFormatter formatter) {
+    // ignores config's DateFormatSupplier and substitutes its own
+    formatter.initialize(data.entrySet(), config);
+
+    assertTrue(formatter.hasNext());
+    final String next = formatter.next();
+    assertTrue(next, next.endsWith("1970/01/01 00:00:00.000"));
+  }
+
+  @Test
+  public void testTimestamps() {
+    final TimeZone utc = TimeZone.getTimeZone("UTC");
+    final TimeZone est = TimeZone.getTimeZone("EST");
+    final FormatterConfig config = new 
FormatterConfig().setPrintTimestamps(true);
+    DateStringFormatter formatter;
+
+    formatter = new DateStringFormatter(utc);
+    testFormatterIgnoresConfig(config, formatter);
+
+    // even though config says to use EST and only print year, the Formatter 
will override these
+    formatter = new DateStringFormatter(utc);
+    DateFormatSupplier dfSupplier = 
DateFormatSupplier.createSimpleFormatSupplier("YYYY", est);
+    config.setDateFormatSupplier(dfSupplier);
+    testFormatterIgnoresConfig(config, formatter);
+  }
+
+  @Test
+  public void testNoTimestamps() {
+    data.put(new Key("", "", "", 1), new Value());
+
+    assertEquals(2, data.size());
+
+    formatter.initialize(data.entrySet(), new FormatterConfig());
+
+    assertEquals(formatter.next(), formatter.next());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
----------------------------------------------------------------------
diff --git 
a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
 
b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
index 47ba1c9..1baa3a1 100644
--- 
a/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
+++ 
b/minicluster/src/main/java/org/apache/accumulo/cluster/standalone/StandaloneAccumuloCluster.java
@@ -38,11 +38,15 @@ import org.apache.accumulo.minicluster.ServerType;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * AccumuloCluster implementation to connect to an existing deployment of 
Accumulo
  */
 public class StandaloneAccumuloCluster implements AccumuloCluster {
+  @SuppressWarnings("unused")
+  private static final Logger log = 
LoggerFactory.getLogger(StandaloneAccumuloCluster.class);
 
   static final List<ServerType> ALL_SERVER_TYPES = 
Collections.unmodifiableList(Arrays.asList(ServerType.MASTER, 
ServerType.TABLET_SERVER, ServerType.TRACER,
       ServerType.GARBAGE_COLLECTOR, ServerType.MONITOR));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
----------------------------------------------------------------------
diff --git 
a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
 
b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
index 1e5a4f9..62b977d 100644
--- 
a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
+++ 
b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
@@ -380,11 +380,9 @@ public class MiniAccumuloClusterImpl implements 
AccumuloCluster {
    * @param config
    *          initial configuration
    */
+  @SuppressWarnings("deprecation")
   public MiniAccumuloClusterImpl(MiniAccumuloConfigImpl config) throws 
IOException {
-    @SuppressWarnings("deprecation")
-    Property INSTANCE_DFS_DIR = Property.INSTANCE_DFS_DIR;
-    @SuppressWarnings("deprecation")
-    Property INSTANCE_DFS_URI = Property.INSTANCE_DFS_URI;
+
     this.config = config.initialize();
 
     mkdirs(config.getConfDir());
@@ -429,8 +427,8 @@ public class MiniAccumuloClusterImpl implements 
AccumuloCluster {
       writeConfig(hdfsFile, conf);
 
       Map<String,String> siteConfig = config.getSiteConfig();
-      siteConfig.put(INSTANCE_DFS_URI.getKey(), dfsUri);
-      siteConfig.put(INSTANCE_DFS_DIR.getKey(), "/accumulo");
+      siteConfig.put(Property.INSTANCE_DFS_URI.getKey(), dfsUri);
+      siteConfig.put(Property.INSTANCE_DFS_DIR.getKey(), "/accumulo");
       config.setSiteConfig(siteConfig);
     } else if (config.useExistingInstance()) {
       dfsUri = 
CachedConfiguration.getInstance().get(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 25dfd46..c2bd679 100644
--- a/pom.xml
+++ b/pom.xml
@@ -619,36 +619,6 @@
         </plugin>
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-invoker-plugin</artifactId>
-          <!-- overridden version from ASF-17 parent pom -->
-          <version>2.0.0</version>
-        </plugin>
-        <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-source-plugin</artifactId>
-          <!-- overridden version from ASF-17 parent pom -->
-          <version>3.0.0</version>
-        </plugin>
-        <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-dependency-plugin</artifactId>
-          <!-- overridden version from ASF-17 parent pom -->
-          <version>2.10</version>
-        </plugin>
-        <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-gpg-plugin</artifactId>
-          <!-- overridden version from ASF-17 parent pom -->
-          <version>1.6</version>
-        </plugin>
-        <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
-          <artifactId>maven-scm-plugin</artifactId>
-          <!-- overridden version from ASF-17 parent pom -->
-          <version>1.9.4</version>
-        </plugin>
-        <plugin>
-          <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-changes-plugin</artifactId>
           <version>2.12</version>
           <configuration>
@@ -664,8 +634,6 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-clean-plugin</artifactId>
-          <!-- overridden version from ASF-17 parent pom -->
-          <version>3.0.0</version>
           <configuration>
             <filesets>
               <fileset>
@@ -681,8 +649,6 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-compiler-plugin</artifactId>
-          <!-- overridden version from ASF-17 parent pom -->
-          <version>3.5.1</version>
           <configuration>
             <optimize>true</optimize>
             <showDeprecation>true</showDeprecation>
@@ -698,8 +664,6 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-jar-plugin</artifactId>
-          <!-- overridden version from ASF-17 parent pom -->
-          <version>2.6</version>
           <configuration>
             <archive>
               <manifestEntries>
@@ -712,8 +676,6 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-javadoc-plugin</artifactId>
-          <!-- overridden version from ASF-17 parent pom -->
-          <version>2.10.3</version>
           <configuration>
             <quiet>true</quiet>
             <javadocVersion>${maven.compiler.target}</javadocVersion>
@@ -724,8 +686,6 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-release-plugin</artifactId>
-          <!-- overridden version from ASF-17 parent pom -->
-          <version>2.5.3</version>
           <configuration>
             <arguments>-P !autoformat,thrift,sunny -Dtimeout.factor=2 
${extraReleaseArgs}</arguments>
             <autoVersionSubmodules>true</autoVersionSubmodules>
@@ -741,8 +701,6 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-site-plugin</artifactId>
-          <!-- overridden version from ASF-17 parent pom -->
-          <version>3.5.1</version>
           <configuration>
             <skipDeploy>true</skipDeploy>
           </configuration>
@@ -750,8 +708,6 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-surefire-plugin</artifactId>
-          <!-- overridden version from ASF-17 parent pom -->
-          <version>2.19.1</version>
           <configuration>
             <systemPropertyVariables>
               <java.io.tmpdir>${project.build.directory}</java.io.tmpdir>
@@ -762,8 +718,6 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-failsafe-plugin</artifactId>
-          <!-- overridden version from ASF-17 parent pom -->
-          <version>2.19.1</version>
           <configuration>
             <systemPropertyVariables>
               <java.io.tmpdir>${project.build.directory}</java.io.tmpdir>
@@ -805,8 +759,6 @@
         <plugin>
           <groupId>org.apache.maven.plugins</groupId>
           <artifactId>maven-enforcer-plugin</artifactId>
-          <!-- overridden version from ASF-17 parent pom -->
-          <version>1.4.1</version>
           <configuration>
             <rules>
               <requireJavaVersion>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/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 c4b422c..56866c2 100644
--- a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
@@ -85,6 +85,7 @@ import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.util.ByteBufferUtil;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.proxy.thrift.AccumuloProxy;
 import org.apache.accumulo.proxy.thrift.BatchScanOptions;
@@ -189,18 +190,23 @@ public class ProxyServer implements AccumuloProxy.Iface {
 
   public ProxyServer(Properties props) {
 
-    ClientConfiguration clientConf;
-    if (props.containsKey("clientConfigurationFile")) {
-      String clientConfFile = props.getProperty("clientConfigurationFile");
-      try {
-        clientConf = new ClientConfiguration(clientConfFile);
-      } catch (ConfigurationException e) {
-        throw new RuntimeException(e);
+    String useMock = props.getProperty("useMockInstance");
+    if (useMock != null && Boolean.parseBoolean(useMock))
+      instance = DeprecationUtil.makeMockInstance(this.getClass().getName());
+    else {
+      ClientConfiguration clientConf;
+      if (props.containsKey("clientConfigurationFile")) {
+        String clientConfFile = props.getProperty("clientConfigurationFile");
+        try {
+          clientConf = new ClientConfiguration(clientConfFile);
+        } catch (ConfigurationException e) {
+          throw new RuntimeException(e);
+        }
+      } else {
+        clientConf = ClientConfiguration.loadDefault();
       }
-    } else {
-      clientConf = ClientConfiguration.loadDefault();
+      instance = new 
ZooKeeperInstance(clientConf.withInstance(props.getProperty("instance")).withZkHosts(props.getProperty("zookeepers")));
     }
-    instance = new 
ZooKeeperInstance(clientConf.withInstance(props.getProperty("instance")).withZkHosts(props.getProperty("zookeepers")));
 
     try {
       String tokenProp = props.getProperty("tokenClass", 
PasswordToken.class.getName());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/server/base/src/main/java/org/apache/accumulo/server/AccumuloServerContext.java
----------------------------------------------------------------------
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/AccumuloServerContext.java
 
b/server/base/src/main/java/org/apache/accumulo/server/AccumuloServerContext.java
index 1a61707..ce7bfad 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/AccumuloServerContext.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/AccumuloServerContext.java
@@ -28,9 +28,11 @@ import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.ConnectorImpl;
 import org.apache.accumulo.core.client.impl.Credentials;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.rpc.SslConnectionParams;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.rpc.SaslServerConnectionParams;
@@ -92,6 +94,9 @@ public class AccumuloServerContext extends ClientContext {
    * Get the credentials to use for this instance so it can be passed to the 
superclass during construction.
    */
   private static Credentials getCredentials(Instance instance) {
+    if (DeprecationUtil.isMockInstance(instance)) {
+      return new Credentials("mockSystemUser", new 
PasswordToken("mockSystemPassword"));
+    }
     return SystemCredentials.get(instance);
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java
----------------------------------------------------------------------
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java
 
b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java
index 7af978b..a058660 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnDefaultTable.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.server.cli;
 
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 
 public class ClientOnDefaultTable extends 
org.apache.accumulo.core.cli.ClientOnDefaultTable {
@@ -30,6 +31,8 @@ public class ClientOnDefaultTable extends 
org.apache.accumulo.core.cli.ClientOnD
     if (cachedInstance != null)
       return cachedInstance;
 
+    if (mock)
+      return cachedInstance = DeprecationUtil.makeMockInstance(instance);
     if (instance == null) {
       return cachedInstance = HdfsZooInstance.getInstance();
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java
----------------------------------------------------------------------
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java
 
b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java
index c966723..e02dd93 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOnRequiredTable.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.server.cli;
 
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 
 public class ClientOnRequiredTable extends 
org.apache.accumulo.core.cli.ClientOnRequiredTable {
@@ -30,6 +31,8 @@ public class ClientOnRequiredTable extends 
org.apache.accumulo.core.cli.ClientOn
     if (cachedInstance != null)
       return cachedInstance;
 
+    if (mock)
+      return cachedInstance = DeprecationUtil.makeMockInstance(instance);
     if (instance == null) {
       return cachedInstance = HdfsZooInstance.getInstance();
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java
----------------------------------------------------------------------
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java 
b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java
index 81a42f8..c91471e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/cli/ClientOpts.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.server.cli;
 
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 
 public class ClientOpts extends org.apache.accumulo.core.cli.ClientOpts {
@@ -28,6 +29,8 @@ public class ClientOpts extends 
org.apache.accumulo.core.cli.ClientOpts {
 
   @Override
   public Instance getInstance() {
+    if (mock)
+      return DeprecationUtil.makeMockInstance(instance);
     if (instance == null) {
       return HdfsZooInstance.getInstance();
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/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 bca8ddf..e4e73d2 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
@@ -19,6 +19,7 @@ package org.apache.accumulo.server.client;
 import static java.nio.charset.StandardCharsets.UTF_8;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
@@ -34,24 +35,28 @@ import org.apache.accumulo.core.client.impl.ConnectorImpl;
 import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.impl.InstanceOperationsImpl;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.OpTimer;
+import org.apache.accumulo.core.util.TextUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.accumulo.server.Accumulo;
+import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import com.google.common.base.Joiner;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Joiner;
-
 /**
  * An implementation of Instance that looks in HDFS and ZooKeeper to find the 
master and root tablet location.
  *
@@ -172,6 +177,38 @@ public class HdfsZooInstance implements Instance {
     return new ConnectorImpl(new ClientContext(this, new 
Credentials(principal, token), SiteConfiguration.getInstance()));
   }
 
+  @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() {
+    return conf = conf == null ? new 
ServerConfigurationFactory(this).getConfiguration() : 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());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/server/base/src/main/java/org/apache/accumulo/server/security/UserImpersonation.java
----------------------------------------------------------------------
diff --git 
a/server/base/src/main/java/org/apache/accumulo/server/security/UserImpersonation.java
 
b/server/base/src/main/java/org/apache/accumulo/server/security/UserImpersonation.java
index 9e8f576..97bc858 100644
--- 
a/server/base/src/main/java/org/apache/accumulo/server/security/UserImpersonation.java
+++ 
b/server/base/src/main/java/org/apache/accumulo/server/security/UserImpersonation.java
@@ -22,11 +22,14 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * When SASL is enabled, this parses properties from the site configuration to 
build up a set of all users capable of impersonating another user, the users
@@ -41,8 +44,9 @@ import org.apache.commons.lang.StringUtils;
  */
 public class UserImpersonation {
 
+  private static final Logger log = 
LoggerFactory.getLogger(UserImpersonation.class);
   private static final Set<String> ALWAYS_TRUE = new AlwaysTrueSet<>();
-  private static final String ALL = "*";
+  private static final String ALL = "*", USERS = "users", HOSTS = "hosts";
 
   public static class AlwaysTrueSet<T> implements Set<T> {
 
@@ -169,6 +173,7 @@ public class UserImpersonation {
 
   private final Map<String,UsersWithHosts> proxyUsers;
 
+  @SuppressWarnings("deprecation")
   public UserImpersonation(AccumuloConfiguration conf) {
     proxyUsers = new HashMap<>();
 
@@ -177,6 +182,9 @@ public class UserImpersonation {
     if 
(!Property.INSTANCE_RPC_SASL_ALLOWED_USER_IMPERSONATION.getDefaultValue().equals(userConfig))
 {
       String hostConfig = 
conf.get(Property.INSTANCE_RPC_SASL_ALLOWED_HOST_IMPERSONATION);
       parseOnelineConfiguration(userConfig, hostConfig);
+    } else {
+      // Otherwise, assume the old-style
+      
parseMultiPropertyConfiguration(conf.getAllPropertiesWithPrefix(Property.INSTANCE_RPC_SASL_PROXYUSERS));
     }
   }
 
@@ -244,6 +252,64 @@ public class UserImpersonation {
     }
   }
 
+  /**
+   * Parses all properties that start with {@link 
Property#INSTANCE_RPC_SASL_PROXYUSERS}. This approach was the original 
configuration method, but does not work
+   * with Ambari.
+   *
+   * @param configProperties
+   *          The relevant configuration properties for impersonation.
+   */
+  @SuppressWarnings("javadoc")
+  private void parseMultiPropertyConfiguration(Map<String,String> 
configProperties) {
+    @SuppressWarnings("deprecation")
+    final String configKey = Property.INSTANCE_RPC_SASL_PROXYUSERS.getKey();
+    for (Entry<String,String> entry : configProperties.entrySet()) {
+      String aclKey = entry.getKey().substring(configKey.length());
+      int index = aclKey.lastIndexOf('.');
+
+      if (-1 == index) {
+        throw new RuntimeException("Expected 2 elements in key suffix: " + 
aclKey);
+      }
+
+      final String remoteUser = aclKey.substring(0, index).trim(), 
usersOrHosts = aclKey.substring(index + 1).trim();
+      UsersWithHosts usersWithHosts = proxyUsers.get(remoteUser);
+      if (null == usersWithHosts) {
+        usersWithHosts = new UsersWithHosts();
+        proxyUsers.put(remoteUser, usersWithHosts);
+      }
+
+      if (USERS.equals(usersOrHosts)) {
+        String userString = entry.getValue().trim();
+        if (ALL.equals(userString)) {
+          usersWithHosts.setAcceptAllUsers(true);
+        } else if (!usersWithHosts.acceptsAllUsers()) {
+          Set<String> users = usersWithHosts.getUsers();
+          if (null == users) {
+            users = new HashSet<>();
+            usersWithHosts.setUsers(users);
+          }
+          String[] userValues = StringUtils.split(userString, ',');
+          users.addAll(Arrays.<String> asList(userValues));
+        }
+      } else if (HOSTS.equals(usersOrHosts)) {
+        String hostsString = entry.getValue().trim();
+        if (ALL.equals(hostsString)) {
+          usersWithHosts.setAcceptAllHosts(true);
+        } else if (!usersWithHosts.acceptsAllHosts()) {
+          Set<String> hosts = usersWithHosts.getHosts();
+          if (null == hosts) {
+            hosts = new HashSet<>();
+            usersWithHosts.setHosts(hosts);
+          }
+          String[] hostValues = StringUtils.split(hostsString, ',');
+          hosts.addAll(Arrays.<String> asList(hostValues));
+        }
+      } else {
+        log.debug("Ignoring key " + aclKey);
+      }
+    }
+  }
+
   public UsersWithHosts get(String remoteUser) {
     return proxyUsers.get(remoteUser);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/server/base/src/test/java/org/apache/accumulo/server/init/InitializeTest.java
----------------------------------------------------------------------
diff --git 
a/server/base/src/test/java/org/apache/accumulo/server/init/InitializeTest.java 
b/server/base/src/test/java/org/apache/accumulo/server/init/InitializeTest.java
index 1f915c0..cb34fb9 100644
--- 
a/server/base/src/test/java/org/apache/accumulo/server/init/InitializeTest.java
+++ 
b/server/base/src/test/java/org/apache/accumulo/server/init/InitializeTest.java
@@ -41,11 +41,6 @@ import org.junit.Test;
  * This test is not thread-safe.
  */
 public class InitializeTest {
-  @SuppressWarnings("deprecation")
-  private static Property INSTANCE_DFS_DIR = Property.INSTANCE_DFS_DIR;
-  @SuppressWarnings("deprecation")
-  private static Property INSTANCE_DFS_URI = Property.INSTANCE_DFS_URI;
-
   private Configuration conf;
   private VolumeManager fs;
   private SiteConfiguration sconf;
@@ -82,9 +77,10 @@ public class InitializeTest {
     assertTrue(Initialize.isInitialized(fs));
   }
 
+  @SuppressWarnings("deprecation")
   @Test
   public void testCheckInit_NoZK() throws Exception {
-    expect(sconf.get(INSTANCE_DFS_URI)).andReturn("hdfs://foo");
+    expect(sconf.get(Property.INSTANCE_DFS_URI)).andReturn("hdfs://foo");
     expectLastCall().anyTimes();
     expect(sconf.get(Property.INSTANCE_ZK_HOST)).andReturn("zk1");
     replay(sconf);
@@ -94,11 +90,12 @@ public class InitializeTest {
     assertFalse(Initialize.checkInit(conf, fs, sconf));
   }
 
+  @SuppressWarnings("deprecation")
   @Test
   public void testCheckInit_AlreadyInit() throws Exception {
-    expect(sconf.get(INSTANCE_DFS_URI)).andReturn("hdfs://foo");
+    expect(sconf.get(Property.INSTANCE_DFS_URI)).andReturn("hdfs://foo");
     expectLastCall().anyTimes();
-    expect(sconf.get(INSTANCE_DFS_DIR)).andReturn("/bar");
+    expect(sconf.get(Property.INSTANCE_DFS_DIR)).andReturn("/bar");
     expect(sconf.get(Property.INSTANCE_VOLUMES)).andReturn("");
     expect(sconf.get(Property.INSTANCE_ZK_HOST)).andReturn("zk1");
     
expect(sconf.get(Property.INSTANCE_SECRET)).andReturn(Property.INSTANCE_SECRET.getDefaultValue());
@@ -112,12 +109,13 @@ public class InitializeTest {
   }
 
   // Cannot test, need to mock static FileSystem.getDefaultUri()
+  @SuppressWarnings("deprecation")
   @Ignore
   @Test
   public void testCheckInit_AlreadyInit_DefaultUri() throws Exception {
-    expect(sconf.get(INSTANCE_DFS_URI)).andReturn("");
+    expect(sconf.get(Property.INSTANCE_DFS_URI)).andReturn("");
     expectLastCall().anyTimes();
-    expect(sconf.get(INSTANCE_DFS_DIR)).andReturn("/bar");
+    expect(sconf.get(Property.INSTANCE_DFS_DIR)).andReturn("/bar");
     expect(sconf.get(Property.INSTANCE_ZK_HOST)).andReturn("zk1");
     
expect(sconf.get(Property.INSTANCE_SECRET)).andReturn(Property.INSTANCE_SECRET.getDefaultValue());
     replay(sconf);
@@ -130,9 +128,10 @@ public class InitializeTest {
     assertFalse(Initialize.checkInit(conf, fs, sconf));
   }
 
+  @SuppressWarnings("deprecation")
   @Test(expected = IOException.class)
   public void testCheckInit_FSException() throws Exception {
-    expect(sconf.get(INSTANCE_DFS_URI)).andReturn("hdfs://foo");
+    expect(sconf.get(Property.INSTANCE_DFS_URI)).andReturn("hdfs://foo");
     expectLastCall().anyTimes();
     expect(sconf.get(Property.INSTANCE_ZK_HOST)).andReturn("zk1");
     
expect(sconf.get(Property.INSTANCE_SECRET)).andReturn(Property.INSTANCE_SECRET.getDefaultValue());
@@ -145,9 +144,10 @@ public class InitializeTest {
     Initialize.checkInit(conf, fs, sconf);
   }
 
+  @SuppressWarnings("deprecation")
   @Test
   public void testCheckInit_OK() throws Exception {
-    expect(sconf.get(INSTANCE_DFS_URI)).andReturn("hdfs://foo");
+    expect(sconf.get(Property.INSTANCE_DFS_URI)).andReturn("hdfs://foo");
     expectLastCall().anyTimes();
     expect(sconf.get(Property.INSTANCE_ZK_HOST)).andReturn("zk1");
     
expect(sconf.get(Property.INSTANCE_SECRET)).andReturn(Property.INSTANCE_SECRET.getDefaultValue());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4f2e6472/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
----------------------------------------------------------------------
diff --git 
a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
 
b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
index b887f29..3ed6a1b 100644
--- 
a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
+++ 
b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
@@ -17,6 +17,7 @@
 package org.apache.accumulo.server.master.balancer;
 
 import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -79,6 +80,34 @@ public abstract class BaseHostRegexTableLoadBalancerTest 
extends HostRegexTableL
       return 30;
     }
 
+    @Deprecated
+    @Override
+    public Connector getConnector(String user, byte[] pass) throws 
AccumuloException, AccumuloSecurityException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Deprecated
+    @Override
+    public Connector getConnector(String user, ByteBuffer pass) throws 
AccumuloException, AccumuloSecurityException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Deprecated
+    @Override
+    public Connector getConnector(String user, CharSequence pass) throws 
AccumuloException, AccumuloSecurityException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Deprecated
+    @Override
+    public AccumuloConfiguration getConfiguration() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Deprecated
+    @Override
+    public void setConfiguration(AccumuloConfiguration conf) {}
+
     @Override
     public Connector getConnector(String principal, AuthenticationToken token) 
throws AccumuloException, AccumuloSecurityException {
       throw new UnsupportedOperationException();

Reply via email to