You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ct...@apache.org on 2015/07/30 23:51:35 UTC

[01/14] accumulo git commit: ACCUMULO-3920 Convert more tests from mock

Repository: accumulo
Updated Branches:
  refs/heads/master 650322351 -> a9e6d7348


http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/replication/WorkMakerIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/WorkMakerIT.java b/test/src/main/java/org/apache/accumulo/test/replication/WorkMakerIT.java
new file mode 100644
index 0000000..6e2c833
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/replication/WorkMakerIT.java
@@ -0,0 +1,209 @@
+/*
+ * 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.test.replication;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
+import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
+import org.apache.accumulo.core.replication.ReplicationTable;
+import org.apache.accumulo.core.replication.ReplicationTarget;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.master.replication.WorkMaker;
+import org.apache.accumulo.server.replication.StatusUtil;
+import org.apache.accumulo.server.replication.proto.Replication.Status;
+import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+
+public class WorkMakerIT extends ConfigurableMacBase {
+
+  private Connector conn;
+
+  private static class MockWorkMaker extends WorkMaker {
+
+    public MockWorkMaker(Connector conn) {
+      super(null, conn);
+    }
+
+    @Override
+    public void setBatchWriter(BatchWriter bw) {
+      super.setBatchWriter(bw);
+    }
+
+    @Override
+    public void addWorkRecord(Text file, Value v, Map<String,String> targets, String sourceTableId) {
+      super.addWorkRecord(file, v, targets, sourceTableId);
+    }
+
+    @Override
+    public boolean shouldCreateWork(Status status) {
+      return super.shouldCreateWork(status);
+    }
+
+  }
+
+  @Before
+  public void setupInstance() throws Exception {
+    conn = getConnector();
+    ReplicationTable.setOnline(conn);
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.WRITE);
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.READ);
+  }
+
+  @Test
+  public void singleUnitSingleTarget() throws Exception {
+    String table = testName.getMethodName();
+    conn.tableOperations().create(table);
+    String tableId = conn.tableOperations().tableIdMap().get(table);
+    String file = "hdfs://localhost:8020/accumulo/wal/123456-1234-1234-12345678";
+
+    // Create a status record for a file
+    long timeCreated = System.currentTimeMillis();
+    Mutation m = new Mutation(new Path(file).toString());
+    m.put(StatusSection.NAME, new Text(tableId), StatusUtil.fileCreatedValue(timeCreated));
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    bw.addMutation(m);
+    bw.flush();
+
+    // Assert that we have one record in the status section
+    Scanner s = ReplicationTable.getScanner(conn);
+    StatusSection.limit(s);
+    Assert.assertEquals(1, Iterables.size(s));
+
+    MockWorkMaker workMaker = new MockWorkMaker(conn);
+
+    // Invoke the addWorkRecord method to create a Work record from the Status record earlier
+    ReplicationTarget expected = new ReplicationTarget("remote_cluster_1", "4", tableId);
+    workMaker.setBatchWriter(bw);
+    workMaker.addWorkRecord(new Text(file), StatusUtil.fileCreatedValue(timeCreated), ImmutableMap.of("remote_cluster_1", "4"), tableId);
+
+    // Scan over just the WorkSection
+    s = ReplicationTable.getScanner(conn);
+    WorkSection.limit(s);
+
+    Entry<Key,Value> workEntry = Iterables.getOnlyElement(s);
+    Key workKey = workEntry.getKey();
+    ReplicationTarget actual = ReplicationTarget.from(workKey.getColumnQualifier());
+
+    Assert.assertEquals(file, workKey.getRow().toString());
+    Assert.assertEquals(WorkSection.NAME, workKey.getColumnFamily());
+    Assert.assertEquals(expected, actual);
+    Assert.assertEquals(workEntry.getValue(), StatusUtil.fileCreatedValue(timeCreated));
+  }
+
+  @Test
+  public void singleUnitMultipleTargets() throws Exception {
+    String table = testName.getMethodName();
+    conn.tableOperations().create(table);
+
+    String tableId = conn.tableOperations().tableIdMap().get(table);
+
+    String file = "hdfs://localhost:8020/accumulo/wal/123456-1234-1234-12345678";
+
+    Mutation m = new Mutation(new Path(file).toString());
+    m.put(StatusSection.NAME, new Text(tableId), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    bw.addMutation(m);
+    bw.flush();
+
+    // Assert that we have one record in the status section
+    Scanner s = ReplicationTable.getScanner(conn);
+    StatusSection.limit(s);
+    Assert.assertEquals(1, Iterables.size(s));
+
+    MockWorkMaker workMaker = new MockWorkMaker(conn);
+
+    Map<String,String> targetClusters = ImmutableMap.of("remote_cluster_1", "4", "remote_cluster_2", "6", "remote_cluster_3", "8");
+    Set<ReplicationTarget> expectedTargets = new HashSet<>();
+    for (Entry<String,String> cluster : targetClusters.entrySet()) {
+      expectedTargets.add(new ReplicationTarget(cluster.getKey(), cluster.getValue(), tableId));
+    }
+    workMaker.setBatchWriter(bw);
+    workMaker.addWorkRecord(new Text(file), StatusUtil.fileCreatedValue(System.currentTimeMillis()), targetClusters, tableId);
+
+    s = ReplicationTable.getScanner(conn);
+    WorkSection.limit(s);
+
+    Set<ReplicationTarget> actualTargets = new HashSet<>();
+    for (Entry<Key,Value> entry : s) {
+      Assert.assertEquals(file, entry.getKey().getRow().toString());
+      Assert.assertEquals(WorkSection.NAME, entry.getKey().getColumnFamily());
+
+      ReplicationTarget target = ReplicationTarget.from(entry.getKey().getColumnQualifier());
+      actualTargets.add(target);
+    }
+
+    for (ReplicationTarget expected : expectedTargets) {
+      Assert.assertTrue("Did not find expected target: " + expected, actualTargets.contains(expected));
+      actualTargets.remove(expected);
+    }
+
+    Assert.assertTrue("Found extra replication work entries: " + actualTargets, actualTargets.isEmpty());
+  }
+
+  @Test
+  public void dontCreateWorkForEntriesWithNothingToReplicate() throws Exception {
+    String table = testName.getMethodName();
+    conn.tableOperations().create(table);
+    String tableId = conn.tableOperations().tableIdMap().get(table);
+    String file = "hdfs://localhost:8020/accumulo/wal/123456-1234-1234-12345678";
+
+    Mutation m = new Mutation(new Path(file).toString());
+    m.put(StatusSection.NAME, new Text(tableId), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    bw.addMutation(m);
+    bw.flush();
+
+    // Assert that we have one record in the status section
+    Scanner s = ReplicationTable.getScanner(conn);
+    StatusSection.limit(s);
+    Assert.assertEquals(1, Iterables.size(s));
+
+    MockWorkMaker workMaker = new MockWorkMaker(conn);
+
+    conn.tableOperations().setProperty(ReplicationTable.NAME, Property.TABLE_REPLICATION_TARGET.getKey() + "remote_cluster_1", "4");
+
+    workMaker.setBatchWriter(bw);
+
+    // If we don't shortcircuit out, we should get an exception because ServerConfiguration.getTableConfiguration
+    // won't work with MockAccumulo
+    workMaker.run();
+
+    s = ReplicationTable.getScanner(conn);
+    WorkSection.limit(s);
+
+    Assert.assertEquals(0, Iterables.size(s));
+  }
+
+}


[02/14] accumulo git commit: ACCUMULO-3920 Convert more tests from mock

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/master/MergeStateIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/master/MergeStateIT.java b/test/src/main/java/org/apache/accumulo/test/master/MergeStateIT.java
new file mode 100644
index 0000000..fb702a2
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/master/MergeStateIT.java
@@ -0,0 +1,204 @@
+/*
+ * 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.test.master;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Set;
+
+import org.apache.accumulo.core.client.BatchDeleter;
+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.MutationsRejectedException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+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.data.impl.KeyExtent;
+import org.apache.accumulo.core.master.thrift.MasterState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ChoppedColumnFamily;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.master.state.MergeStats;
+import org.apache.accumulo.server.AccumuloServerContext;
+import org.apache.accumulo.server.master.state.Assignment;
+import org.apache.accumulo.server.master.state.CurrentState;
+import org.apache.accumulo.server.master.state.MergeInfo;
+import org.apache.accumulo.server.master.state.MergeState;
+import org.apache.accumulo.server.master.state.MetaDataStateStore;
+import org.apache.accumulo.server.master.state.TServerInstance;
+import org.apache.accumulo.server.master.state.TabletLocationState;
+import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.apache.hadoop.io.Text;
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.net.HostAndPort;
+
+public class MergeStateIT extends ConfigurableMacBase {
+
+  private static class MockCurrentState implements CurrentState {
+
+    TServerInstance someTServer = new TServerInstance(HostAndPort.fromParts("127.0.0.1", 1234), 0x123456);
+    MergeInfo mergeInfo;
+
+    MockCurrentState(MergeInfo info) {
+      this.mergeInfo = info;
+    }
+
+    @Override
+    public Set<String> onlineTables() {
+      return Collections.singleton("t");
+    }
+
+    @Override
+    public Set<TServerInstance> onlineTabletServers() {
+      return Collections.singleton(someTServer);
+    }
+
+    @Override
+    public Collection<MergeInfo> merges() {
+      return Collections.singleton(mergeInfo);
+    }
+
+    @Override
+    public Collection<KeyExtent> migrations() {
+      return Collections.emptyList();
+    }
+
+    @Override
+    public MasterState getMasterState() {
+      return MasterState.NORMAL;
+    }
+
+    @Override
+    public Set<TServerInstance> shutdownServers() {
+      return Collections.emptySet();
+    }
+  }
+
+  private static void update(Connector c, Mutation m) throws TableNotFoundException, MutationsRejectedException {
+    BatchWriter bw = c.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+    bw.addMutation(m);
+    bw.close();
+  }
+
+  @Test
+  public void test() throws Exception {
+    AccumuloServerContext context = EasyMock.createMock(AccumuloServerContext.class);
+    Connector connector = getConnector();
+    EasyMock.expect(context.getConnector()).andReturn(connector).anyTimes();
+    EasyMock.replay(context);
+    connector.securityOperations().grantTablePermission(connector.whoami(), MetadataTable.NAME, TablePermission.WRITE);
+    BatchWriter bw = connector.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+
+    // Create a fake METADATA table with these splits
+    String splits[] = {"a", "e", "j", "o", "t", "z"};
+    // create metadata for a table "t" with the splits above
+    Text tableId = new Text("t");
+    Text pr = null;
+    for (String s : splits) {
+      Text split = new Text(s);
+      Mutation prevRow = KeyExtent.getPrevRowUpdateMutation(new KeyExtent(tableId, split, pr));
+      prevRow.put(TabletsSection.CurrentLocationColumnFamily.NAME, new Text("123456"), new Value("127.0.0.1:1234".getBytes()));
+      ChoppedColumnFamily.CHOPPED_COLUMN.put(prevRow, new Value("junk".getBytes()));
+      bw.addMutation(prevRow);
+      pr = split;
+    }
+    // Add the default tablet
+    Mutation defaultTablet = KeyExtent.getPrevRowUpdateMutation(new KeyExtent(tableId, null, pr));
+    defaultTablet.put(TabletsSection.CurrentLocationColumnFamily.NAME, new Text("123456"), new Value("127.0.0.1:1234".getBytes()));
+    bw.addMutation(defaultTablet);
+    bw.close();
+
+    // Read out the TabletLocationStates
+    MockCurrentState state = new MockCurrentState(new MergeInfo(new KeyExtent(tableId, new Text("p"), new Text("e")), MergeInfo.Operation.MERGE));
+
+    // Verify the tablet state: hosted, and count
+    MetaDataStateStore metaDataStateStore = new MetaDataStateStore(context, state);
+    int count = 0;
+    for (TabletLocationState tss : metaDataStateStore) {
+      if (tss != null)
+        count++;
+    }
+    Assert.assertEquals(0, count); // the normal case is to skip tablets in a good state
+
+    // Create the hole
+    // Split the tablet at one end of the range
+    Mutation m = new KeyExtent(tableId, new Text("t"), new Text("p")).getPrevRowUpdateMutation();
+    TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.put(m, new Value("0.5".getBytes()));
+    TabletsSection.TabletColumnFamily.OLD_PREV_ROW_COLUMN.put(m, KeyExtent.encodePrevEndRow(new Text("o")));
+    update(connector, m);
+
+    // do the state check
+    MergeStats stats = scan(state, metaDataStateStore);
+    MergeState newState = stats.nextMergeState(connector, state);
+    Assert.assertEquals(MergeState.WAITING_FOR_OFFLINE, newState);
+
+    // unassign the tablets
+    BatchDeleter deleter = connector.createBatchDeleter(MetadataTable.NAME, Authorizations.EMPTY, 1000, new BatchWriterConfig());
+    deleter.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
+    deleter.setRanges(Collections.singletonList(new Range()));
+    deleter.delete();
+
+    // now we should be ready to merge but, we have inconsistent metadata
+    stats = scan(state, metaDataStateStore);
+    Assert.assertEquals(MergeState.WAITING_FOR_OFFLINE, stats.nextMergeState(connector, state));
+
+    // finish the split
+    KeyExtent tablet = new KeyExtent(tableId, new Text("p"), new Text("o"));
+    m = tablet.getPrevRowUpdateMutation();
+    TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.put(m, new Value("0.5".getBytes()));
+    update(connector, m);
+    metaDataStateStore.setLocations(Collections.singletonList(new Assignment(tablet, state.someTServer)));
+
+    // onos... there's a new tablet online
+    stats = scan(state, metaDataStateStore);
+    Assert.assertEquals(MergeState.WAITING_FOR_CHOPPED, stats.nextMergeState(connector, state));
+
+    // chop it
+    m = tablet.getPrevRowUpdateMutation();
+    ChoppedColumnFamily.CHOPPED_COLUMN.put(m, new Value("junk".getBytes()));
+    update(connector, m);
+
+    stats = scan(state, metaDataStateStore);
+    Assert.assertEquals(MergeState.WAITING_FOR_OFFLINE, stats.nextMergeState(connector, state));
+
+    // take it offline
+    m = tablet.getPrevRowUpdateMutation();
+    Collection<Collection<String>> walogs = Collections.emptyList();
+    metaDataStateStore.unassign(Collections.singletonList(new TabletLocationState(tablet, null, state.someTServer, null, walogs, false)), null);
+
+    // now we can split
+    stats = scan(state, metaDataStateStore);
+    Assert.assertEquals(MergeState.MERGING, stats.nextMergeState(connector, state));
+
+  }
+
+  private MergeStats scan(MockCurrentState state, MetaDataStateStore metaDataStateStore) {
+    MergeStats stats = new MergeStats(state.mergeInfo);
+    stats.getMergeInfo().setState(MergeState.WAITING_FOR_OFFLINE);
+    for (TabletLocationState tss : metaDataStateStore) {
+      stats.update(tss.extent, tss.getState(state.onlineTabletServers()), tss.chopped, false);
+    }
+    return stats;
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/replication/FinishedWorkUpdaterIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/FinishedWorkUpdaterIT.java b/test/src/main/java/org/apache/accumulo/test/replication/FinishedWorkUpdaterIT.java
new file mode 100644
index 0000000..5519013
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/replication/FinishedWorkUpdaterIT.java
@@ -0,0 +1,173 @@
+package org.apache.accumulo.test.replication;
+
+/*
+ * 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.
+ */
+
+import java.util.Map.Entry;
+import java.util.UUID;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+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.protobuf.ProtobufUtil;
+import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
+import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
+import org.apache.accumulo.core.replication.ReplicationTable;
+import org.apache.accumulo.core.replication.ReplicationTarget;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.master.replication.FinishedWorkUpdater;
+import org.apache.accumulo.server.replication.proto.Replication.Status;
+import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.Iterables;
+
+public class FinishedWorkUpdaterIT extends ConfigurableMacBase {
+
+  private Connector conn;
+  private FinishedWorkUpdater updater;
+
+  @Before
+  public void configureUpdater() throws Exception {
+    conn = getConnector();
+    updater = new FinishedWorkUpdater(conn);
+  }
+
+  @Test
+  public void offlineReplicationTableFailsGracefully() {
+    updater.run();
+  }
+
+  @Test
+  public void recordsWithProgressUpdateBothTables() throws Exception {
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.READ);
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.WRITE);
+    ReplicationTable.setOnline(conn);
+
+    String file = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
+    Status stat = Status.newBuilder().setBegin(100).setEnd(200).setClosed(true).setInfiniteEnd(false).build();
+    ReplicationTarget target = new ReplicationTarget("peer", "table1", "1");
+
+    // Create a single work record for a file to some peer
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    Mutation m = new Mutation(file);
+    WorkSection.add(m, target.toText(), ProtobufUtil.toValue(stat));
+    bw.addMutation(m);
+    bw.close();
+
+    updater.run();
+
+    Scanner s = ReplicationTable.getScanner(conn);
+    s.setRange(Range.exact(file));
+    StatusSection.limit(s);
+    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
+
+    Assert.assertEquals(entry.getKey().getColumnFamily(), StatusSection.NAME);
+    Assert.assertEquals(entry.getKey().getColumnQualifier().toString(), target.getSourceTableId());
+
+    // We should only rely on the correct begin attribute being returned
+    Status actual = Status.parseFrom(entry.getValue().get());
+    Assert.assertEquals(stat.getBegin(), actual.getBegin());
+  }
+
+  @Test
+  public void chooseMinimumBeginOffset() throws Exception {
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.READ);
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.WRITE);
+    ReplicationTable.setOnline(conn);
+
+    String file = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
+    // @formatter:off
+    Status stat1 = Status.newBuilder().setBegin(100).setEnd(1000).setClosed(true).setInfiniteEnd(false).build(),
+        stat2 = Status.newBuilder().setBegin(500).setEnd(1000).setClosed(true).setInfiniteEnd(false).build(),
+        stat3 = Status.newBuilder().setBegin(1).setEnd(1000).setClosed(true).setInfiniteEnd(false).build();
+    ReplicationTarget target1 = new ReplicationTarget("peer1", "table1", "1"),
+        target2 = new ReplicationTarget("peer2", "table2", "1"),
+        target3 = new ReplicationTarget("peer3", "table3", "1");
+    // @formatter:on
+
+    // Create a single work record for a file to some peer
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    Mutation m = new Mutation(file);
+    WorkSection.add(m, target1.toText(), ProtobufUtil.toValue(stat1));
+    WorkSection.add(m, target2.toText(), ProtobufUtil.toValue(stat2));
+    WorkSection.add(m, target3.toText(), ProtobufUtil.toValue(stat3));
+    bw.addMutation(m);
+    bw.close();
+
+    updater.run();
+
+    Scanner s = ReplicationTable.getScanner(conn);
+    s.setRange(Range.exact(file));
+    StatusSection.limit(s);
+    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
+
+    Assert.assertEquals(entry.getKey().getColumnFamily(), StatusSection.NAME);
+    Assert.assertEquals(entry.getKey().getColumnQualifier().toString(), target1.getSourceTableId());
+
+    // We should only rely on the correct begin attribute being returned
+    Status actual = Status.parseFrom(entry.getValue().get());
+    Assert.assertEquals(1, actual.getBegin());
+  }
+
+  @Test
+  public void chooseMinimumBeginOffsetInfiniteEnd() throws Exception {
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.READ);
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.WRITE);
+    ReplicationTable.setOnline(conn);
+
+    String file = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
+    // @formatter:off
+    Status stat1 = Status.newBuilder().setBegin(100).setEnd(1000).setClosed(true).setInfiniteEnd(true).build(),
+        stat2 = Status.newBuilder().setBegin(1).setEnd(1000).setClosed(true).setInfiniteEnd(true).build(),
+        stat3 = Status.newBuilder().setBegin(500).setEnd(1000).setClosed(true).setInfiniteEnd(true).build();
+    ReplicationTarget target1 = new ReplicationTarget("peer1", "table1", "1"),
+        target2 = new ReplicationTarget("peer2", "table2", "1"),
+        target3 = new ReplicationTarget("peer3", "table3", "1");
+    // @formatter:on
+
+    // Create a single work record for a file to some peer
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    Mutation m = new Mutation(file);
+    WorkSection.add(m, target1.toText(), ProtobufUtil.toValue(stat1));
+    WorkSection.add(m, target2.toText(), ProtobufUtil.toValue(stat2));
+    WorkSection.add(m, target3.toText(), ProtobufUtil.toValue(stat3));
+    bw.addMutation(m);
+    bw.close();
+
+    updater.run();
+
+    Scanner s = ReplicationTable.getScanner(conn);
+    s.setRange(Range.exact(file));
+    StatusSection.limit(s);
+    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
+
+    Assert.assertEquals(entry.getKey().getColumnFamily(), StatusSection.NAME);
+    Assert.assertEquals(entry.getKey().getColumnQualifier().toString(), target1.getSourceTableId());
+
+    // We should only rely on the correct begin attribute being returned
+    Status actual = Status.parseFrom(entry.getValue().get());
+    Assert.assertEquals(1, actual.getBegin());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/replication/RemoveCompleteReplicationRecordsIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/RemoveCompleteReplicationRecordsIT.java b/test/src/main/java/org/apache/accumulo/test/replication/RemoveCompleteReplicationRecordsIT.java
new file mode 100644
index 0000000..df1f64f
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/replication/RemoveCompleteReplicationRecordsIT.java
@@ -0,0 +1,337 @@
+/*
+ * 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.test.replication;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.accumulo.core.client.BatchScanner;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.IteratorSetting;
+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.user.WholeRowIterator;
+import org.apache.accumulo.core.protobuf.ProtobufUtil;
+import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
+import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
+import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
+import org.apache.accumulo.core.replication.ReplicationTable;
+import org.apache.accumulo.core.replication.ReplicationTarget;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.master.replication.RemoveCompleteReplicationRecords;
+import org.apache.accumulo.server.replication.StatusUtil;
+import org.apache.accumulo.server.replication.proto.Replication.Status;
+import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.apache.hadoop.io.Text;
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.Iterables;
+
+public class RemoveCompleteReplicationRecordsIT extends ConfigurableMacBase {
+
+  private MockRemoveCompleteReplicationRecords rcrr;
+  private Connector conn;
+
+  private static class MockRemoveCompleteReplicationRecords extends RemoveCompleteReplicationRecords {
+
+    public MockRemoveCompleteReplicationRecords(Connector conn) {
+      super(conn);
+    }
+
+    @Override
+    public long removeCompleteRecords(Connector conn, BatchScanner bs, BatchWriter bw) {
+      return super.removeCompleteRecords(conn, bs, bw);
+    }
+
+  }
+
+  @Before
+  public void initialize() throws Exception {
+    conn = getConnector();
+    rcrr = new MockRemoveCompleteReplicationRecords(conn);
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.READ);
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.WRITE);
+    ReplicationTable.setOnline(conn);
+  }
+
+  @Test
+  public void notYetReplicationRecordsIgnored() throws Exception {
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    int numRecords = 3;
+    for (int i = 0; i < numRecords; i++) {
+      String file = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
+      Mutation m = new Mutation(file);
+      StatusSection.add(m, new Text(Integer.toString(i)), StatusUtil.openWithUnknownLengthValue());
+      bw.addMutation(m);
+    }
+
+    bw.close();
+
+    Assert.assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(conn)));
+
+    BatchScanner bs = ReplicationTable.getBatchScanner(conn, 1);
+    bs.setRanges(Collections.singleton(new Range()));
+    IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
+    bs.addScanIterator(cfg);
+    bw = EasyMock.createMock(BatchWriter.class);
+
+    EasyMock.replay(bw);
+
+    rcrr.removeCompleteRecords(conn, bs, bw);
+    bs.close();
+
+    Assert.assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(conn)));
+  }
+
+  @Test
+  public void partiallyReplicatedRecordsIgnored() throws Exception {
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    int numRecords = 3;
+    Status.Builder builder = Status.newBuilder();
+    builder.setClosed(false);
+    builder.setEnd(10000);
+    builder.setInfiniteEnd(false);
+    for (int i = 0; i < numRecords; i++) {
+      String file = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
+      Mutation m = new Mutation(file);
+      StatusSection.add(m, new Text(Integer.toString(i)), ProtobufUtil.toValue(builder.setBegin(1000 * (i + 1)).build()));
+      bw.addMutation(m);
+    }
+
+    bw.close();
+
+    Assert.assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(conn)));
+
+    BatchScanner bs = ReplicationTable.getBatchScanner(conn, 1);
+    bs.setRanges(Collections.singleton(new Range()));
+    IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
+    bs.addScanIterator(cfg);
+    bw = EasyMock.createMock(BatchWriter.class);
+
+    EasyMock.replay(bw);
+
+    // We don't remove any records, so we can just pass in a fake BW for both
+    rcrr.removeCompleteRecords(conn, bs, bw);
+    bs.close();
+
+    Assert.assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(conn)));
+  }
+
+  @Test
+  public void replicatedClosedWorkRecordsAreNotRemovedWithoutClosedStatusRecords() throws Exception {
+    BatchWriter replBw = ReplicationTable.getBatchWriter(conn);
+    int numRecords = 3;
+
+    Status.Builder builder = Status.newBuilder();
+    builder.setClosed(false);
+    builder.setEnd(10000);
+    builder.setInfiniteEnd(false);
+
+    // Write out numRecords entries to both replication and metadata tables, none of which are fully replicated
+    for (int i = 0; i < numRecords; i++) {
+      String file = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
+      Mutation m = new Mutation(file);
+      StatusSection.add(m, new Text(Integer.toString(i)), ProtobufUtil.toValue(builder.setBegin(1000 * (i + 1)).build()));
+      replBw.addMutation(m);
+    }
+
+    // Add two records that we can delete
+    String fileToRemove = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
+    Mutation m = new Mutation(fileToRemove);
+    StatusSection.add(m, new Text("5"), ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(false).build()));
+    replBw.addMutation(m);
+
+    numRecords++;
+
+    fileToRemove = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
+    m = new Mutation(fileToRemove);
+    StatusSection.add(m, new Text("6"), ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(false).build()));
+    replBw.addMutation(m);
+
+    numRecords++;
+
+    replBw.flush();
+
+    // Make sure that we have the expected number of records in both tables
+    Assert.assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(conn)));
+
+    // We should not remove any records because they're missing closed status
+    BatchScanner bs = ReplicationTable.getBatchScanner(conn, 1);
+    bs.setRanges(Collections.singleton(new Range()));
+    IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
+    bs.addScanIterator(cfg);
+
+    try {
+      Assert.assertEquals(0l, rcrr.removeCompleteRecords(conn, bs, replBw));
+    } finally {
+      bs.close();
+      replBw.close();
+    }
+  }
+
+  @Test
+  public void replicatedClosedRowsAreRemoved() throws Exception {
+    BatchWriter replBw = ReplicationTable.getBatchWriter(conn);
+    int numRecords = 3;
+
+    Status.Builder builder = Status.newBuilder();
+    builder.setClosed(false);
+    builder.setEnd(10000);
+    builder.setInfiniteEnd(false);
+
+    long time = System.currentTimeMillis();
+    // Write out numRecords entries to both replication and metadata tables, none of which are fully replicated
+    for (int i = 0; i < numRecords; i++) {
+      builder.setCreatedTime(time++);
+      String file = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
+      Mutation m = new Mutation(file);
+      Value v = ProtobufUtil.toValue(builder.setBegin(1000 * (i + 1)).build());
+      StatusSection.add(m, new Text(Integer.toString(i)), v);
+      replBw.addMutation(m);
+      m = OrderSection.createMutation(file, time);
+      OrderSection.add(m, new Text(Integer.toString(i)), v);
+      replBw.addMutation(m);
+    }
+
+    Set<String> filesToRemove = new HashSet<>();
+    // We created two mutations for each file
+    numRecords *= 2;
+    int finalNumRecords = numRecords;
+
+    // Add two records that we can delete
+    String fileToRemove = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
+    filesToRemove.add(fileToRemove);
+    Mutation m = new Mutation(fileToRemove);
+    ReplicationTarget target = new ReplicationTarget("peer1", "5", "5");
+    Value value = ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(true).setCreatedTime(time).build());
+    StatusSection.add(m, new Text("5"), value);
+    WorkSection.add(m, target.toText(), value);
+    replBw.addMutation(m);
+
+    m = OrderSection.createMutation(fileToRemove, time);
+    OrderSection.add(m, new Text("5"), value);
+    replBw.addMutation(m);
+    time++;
+
+    numRecords += 3;
+
+    fileToRemove = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
+    filesToRemove.add(fileToRemove);
+    m = new Mutation(fileToRemove);
+    value = ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(true).setCreatedTime(time).build());
+    target = new ReplicationTarget("peer1", "6", "6");
+    StatusSection.add(m, new Text("6"), value);
+    WorkSection.add(m, target.toText(), value);
+    replBw.addMutation(m);
+
+    m = OrderSection.createMutation(fileToRemove, time);
+    OrderSection.add(m, new Text("6"), value);
+    replBw.addMutation(m);
+    time++;
+
+    numRecords += 3;
+
+    replBw.flush();
+
+    // Make sure that we have the expected number of records in both tables
+    Assert.assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(conn)));
+
+    // We should remove the two fully completed records we inserted
+    BatchScanner bs = ReplicationTable.getBatchScanner(conn, 1);
+    bs.setRanges(Collections.singleton(new Range()));
+    StatusSection.limit(bs);
+    WorkSection.limit(bs);
+    IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
+    bs.addScanIterator(cfg);
+
+    try {
+      Assert.assertEquals(4l, rcrr.removeCompleteRecords(conn, bs, replBw));
+    } finally {
+      bs.close();
+      replBw.close();
+    }
+
+    int actualRecords = 0;
+    for (Entry<Key,Value> entry : ReplicationTable.getScanner(conn)) {
+      Assert.assertFalse(filesToRemove.contains(entry.getKey().getRow().toString()));
+      actualRecords++;
+    }
+
+    Assert.assertEquals(finalNumRecords, actualRecords);
+  }
+
+  @Test
+  public void partiallyReplicatedEntriesPrecludeRowDeletion() throws Exception {
+    BatchWriter replBw = ReplicationTable.getBatchWriter(conn);
+    int numRecords = 3;
+
+    Status.Builder builder = Status.newBuilder();
+    builder.setClosed(false);
+    builder.setEnd(10000);
+    builder.setInfiniteEnd(false);
+
+    // Write out numRecords entries to both replication and metadata tables, none of which are fully replicated
+    for (int i = 0; i < numRecords; i++) {
+      String file = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
+      Mutation m = new Mutation(file);
+      StatusSection.add(m, new Text(Integer.toString(i)), ProtobufUtil.toValue(builder.setBegin(1000 * (i + 1)).build()));
+      replBw.addMutation(m);
+    }
+
+    // Add two records that we can delete
+    String fileToRemove = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
+    Mutation m = new Mutation(fileToRemove);
+    ReplicationTarget target = new ReplicationTarget("peer1", "5", "5");
+    Value value = ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(true).build());
+    StatusSection.add(m, new Text("5"), value);
+    WorkSection.add(m, target.toText(), value);
+    target = new ReplicationTarget("peer2", "5", "5");
+    WorkSection.add(m, target.toText(), value);
+    target = new ReplicationTarget("peer3", "5", "5");
+    WorkSection.add(m, target.toText(), ProtobufUtil.toValue(builder.setClosed(false).build()));
+    replBw.addMutation(m);
+
+    numRecords += 4;
+
+    replBw.flush();
+
+    // Make sure that we have the expected number of records in both tables
+    Assert.assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(conn)));
+
+    // We should remove the two fully completed records we inserted
+    BatchScanner bs = ReplicationTable.getBatchScanner(conn, 1);
+    bs.setRanges(Collections.singleton(new Range()));
+    IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
+    bs.addScanIterator(cfg);
+
+    try {
+      Assert.assertEquals(0l, rcrr.removeCompleteRecords(conn, bs, replBw));
+    } finally {
+      bs.close();
+      replBw.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/replication/ReplicationOperationsImplIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/ReplicationOperationsImplIT.java b/test/src/main/java/org/apache/accumulo/test/replication/ReplicationOperationsImplIT.java
new file mode 100644
index 0000000..fa026d1
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/replication/ReplicationOperationsImplIT.java
@@ -0,0 +1,447 @@
+/*
+ * 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.test.replication;
+
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.ClientContext;
+import org.apache.accumulo.core.client.impl.Credentials;
+import org.apache.accumulo.core.client.impl.ReplicationOperationsImpl;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+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.Value;
+import org.apache.accumulo.core.data.impl.KeyExtent;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
+import org.apache.accumulo.core.protobuf.ProtobufUtil;
+import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
+import org.apache.accumulo.core.replication.ReplicationTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.security.thrift.TCredentials;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
+import org.apache.accumulo.core.trace.thrift.TInfo;
+import org.apache.accumulo.master.Master;
+import org.apache.accumulo.master.MasterClientServiceHandler;
+import org.apache.accumulo.server.replication.proto.Replication.Status;
+import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.apache.hadoop.io.Text;
+import org.apache.thrift.TException;
+import org.easymock.EasyMock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReplicationOperationsImplIT extends ConfigurableMacBase {
+  private static final Logger log = LoggerFactory.getLogger(ReplicationOperationsImplIT.class);
+
+  private Instance inst;
+  private Connector conn;
+
+  @Before
+  public void configureInstance() throws Exception {
+    conn = getConnector();
+    inst = conn.getInstance();
+    ReplicationTable.setOnline(conn);
+    conn.securityOperations().grantTablePermission(conn.whoami(), MetadataTable.NAME, TablePermission.WRITE);
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.READ);
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.WRITE);
+  }
+
+  /**
+   * Spoof out the Master so we can call the implementation without starting a full instance.
+   */
+  private ReplicationOperationsImpl getReplicationOperations() throws Exception {
+    Master master = EasyMock.createMock(Master.class);
+    EasyMock.expect(master.getConnector()).andReturn(conn).anyTimes();
+    EasyMock.expect(master.getInstance()).andReturn(inst).anyTimes();
+    EasyMock.replay(master);
+
+    final MasterClientServiceHandler mcsh = new MasterClientServiceHandler(master) {
+      @Override
+      protected String getTableId(Instance inst, String tableName) throws ThriftTableOperationException {
+        try {
+          return conn.tableOperations().tableIdMap().get(tableName);
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+      }
+    };
+
+    ClientContext context = new ClientContext(inst, new Credentials("root", new PasswordToken(ROOT_PASSWORD)), getClientConfig());
+    return new ReplicationOperationsImpl(context) {
+      @Override
+      protected boolean getMasterDrain(final TInfo tinfo, final TCredentials rpcCreds, final String tableName, final Set<String> wals)
+          throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+        try {
+          return mcsh.drainReplicationTable(tinfo, rpcCreds, tableName, wals);
+        } catch (TException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    };
+  }
+
+  @Test
+  public void waitsUntilEntriesAreReplicated() throws Exception {
+    conn.tableOperations().create("foo");
+    Text tableId = new Text(conn.tableOperations().tableIdMap().get("foo"));
+
+    String file1 = "/accumulo/wals/tserver+port/" + UUID.randomUUID(), file2 = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
+    Status stat = Status.newBuilder().setBegin(0).setEnd(10000).setInfiniteEnd(false).setClosed(false).build();
+
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+
+    Mutation m = new Mutation(file1);
+    StatusSection.add(m, tableId, ProtobufUtil.toValue(stat));
+    bw.addMutation(m);
+
+    m = new Mutation(file2);
+    StatusSection.add(m, tableId, ProtobufUtil.toValue(stat));
+    bw.addMutation(m);
+
+    bw.close();
+
+    bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
+    m.put(ReplicationSection.COLF, tableId, ProtobufUtil.toValue(stat));
+
+    bw.addMutation(m);
+
+    m = new Mutation(ReplicationSection.getRowPrefix() + file2);
+    m.put(ReplicationSection.COLF, tableId, ProtobufUtil.toValue(stat));
+
+    bw.close();
+
+    final AtomicBoolean done = new AtomicBoolean(false);
+    final AtomicBoolean exception = new AtomicBoolean(false);
+    final ReplicationOperationsImpl roi = getReplicationOperations();
+    Thread t = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          roi.drain("foo");
+        } catch (Exception e) {
+          log.error("Got error", e);
+          exception.set(true);
+        }
+        done.set(true);
+      }
+    });
+
+    t.start();
+
+    // With the records, we shouldn't be drained
+    Assert.assertFalse(done.get());
+
+    bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
+    m.putDelete(ReplicationSection.COLF, tableId);
+    bw.addMutation(m);
+    bw.flush();
+
+    Assert.assertFalse(done.get());
+
+    m = new Mutation(ReplicationSection.getRowPrefix() + file2);
+    m.putDelete(ReplicationSection.COLF, tableId);
+    bw.addMutation(m);
+    bw.flush();
+    bw.close();
+
+    // Removing metadata entries doesn't change anything
+    Assert.assertFalse(done.get());
+
+    // Remove the replication entries too
+    bw = ReplicationTable.getBatchWriter(conn);
+    m = new Mutation(file1);
+    m.putDelete(StatusSection.NAME, tableId);
+    bw.addMutation(m);
+    bw.flush();
+
+    Assert.assertFalse(done.get());
+
+    m = new Mutation(file2);
+    m.putDelete(StatusSection.NAME, tableId);
+    bw.addMutation(m);
+    bw.flush();
+
+    try {
+      t.join(5000);
+    } catch (InterruptedException e) {
+      Assert.fail("ReplicationOperations.drain did not complete");
+    }
+
+    // After both metadata and replication
+    Assert.assertTrue("Drain never finished", done.get());
+    Assert.assertFalse("Saw unexpectetd exception", exception.get());
+  }
+
+  @Test
+  public void unrelatedReplicationRecordsDontBlockDrain() throws Exception {
+    conn.tableOperations().create("foo");
+    conn.tableOperations().create("bar");
+
+    Text tableId1 = new Text(conn.tableOperations().tableIdMap().get("foo"));
+    Text tableId2 = new Text(conn.tableOperations().tableIdMap().get("bar"));
+
+    String file1 = "/accumulo/wals/tserver+port/" + UUID.randomUUID(), file2 = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
+    Status stat = Status.newBuilder().setBegin(0).setEnd(10000).setInfiniteEnd(false).setClosed(false).build();
+
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+
+    Mutation m = new Mutation(file1);
+    StatusSection.add(m, tableId1, ProtobufUtil.toValue(stat));
+    bw.addMutation(m);
+
+    m = new Mutation(file2);
+    StatusSection.add(m, tableId2, ProtobufUtil.toValue(stat));
+    bw.addMutation(m);
+
+    bw.close();
+
+    bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
+    m.put(ReplicationSection.COLF, tableId1, ProtobufUtil.toValue(stat));
+
+    bw.addMutation(m);
+
+    m = new Mutation(ReplicationSection.getRowPrefix() + file2);
+    m.put(ReplicationSection.COLF, tableId2, ProtobufUtil.toValue(stat));
+
+    bw.close();
+
+    final AtomicBoolean done = new AtomicBoolean(false);
+    final AtomicBoolean exception = new AtomicBoolean(false);
+
+    final ReplicationOperationsImpl roi = getReplicationOperations();
+
+    Thread t = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          roi.drain("foo");
+        } catch (Exception e) {
+          log.error("Got error", e);
+          exception.set(true);
+        }
+        done.set(true);
+      }
+    });
+
+    t.start();
+
+    // With the records, we shouldn't be drained
+    Assert.assertFalse(done.get());
+
+    bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
+    m.putDelete(ReplicationSection.COLF, tableId1);
+    bw.addMutation(m);
+    bw.flush();
+
+    // Removing metadata entries doesn't change anything
+    Assert.assertFalse(done.get());
+
+    // Remove the replication entries too
+    bw = ReplicationTable.getBatchWriter(conn);
+    m = new Mutation(file1);
+    m.putDelete(StatusSection.NAME, tableId1);
+    bw.addMutation(m);
+    bw.flush();
+
+    try {
+      t.join(5000);
+    } catch (InterruptedException e) {
+      Assert.fail("ReplicationOperations.drain did not complete");
+    }
+
+    // After both metadata and replication
+    Assert.assertTrue("Drain never completed", done.get());
+    Assert.assertFalse("Saw unexpected exception", exception.get());
+  }
+
+  @Test
+  public void inprogressReplicationRecordsBlockExecution() throws Exception {
+    conn.tableOperations().create("foo");
+
+    Text tableId1 = new Text(conn.tableOperations().tableIdMap().get("foo"));
+
+    String file1 = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
+    Status stat = Status.newBuilder().setBegin(0).setEnd(10000).setInfiniteEnd(false).setClosed(false).build();
+
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+
+    Mutation m = new Mutation(file1);
+    StatusSection.add(m, tableId1, ProtobufUtil.toValue(stat));
+    bw.addMutation(m);
+    bw.close();
+
+    LogEntry logEntry = new LogEntry(new KeyExtent(new Text(tableId1), null, null), System.currentTimeMillis(), "tserver", file1);
+
+    bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
+    m.put(ReplicationSection.COLF, tableId1, ProtobufUtil.toValue(stat));
+    bw.addMutation(m);
+
+    m = new Mutation(logEntry.getRow());
+    m.put(logEntry.getColumnFamily(), logEntry.getColumnQualifier(), logEntry.getValue());
+    bw.addMutation(m);
+
+    bw.close();
+
+    final AtomicBoolean done = new AtomicBoolean(false);
+    final AtomicBoolean exception = new AtomicBoolean(false);
+    final ReplicationOperationsImpl roi = getReplicationOperations();
+    Thread t = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          roi.drain("foo");
+        } catch (Exception e) {
+          log.error("Got error", e);
+          exception.set(true);
+        }
+        done.set(true);
+      }
+    });
+
+    t.start();
+
+    // With the records, we shouldn't be drained
+    Assert.assertFalse(done.get());
+
+    Status newStatus = Status.newBuilder().setBegin(1000).setEnd(2000).setInfiniteEnd(false).setClosed(true).build();
+    bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
+    m.put(ReplicationSection.COLF, tableId1, ProtobufUtil.toValue(newStatus));
+    bw.addMutation(m);
+    bw.flush();
+
+    // Removing metadata entries doesn't change anything
+    Assert.assertFalse(done.get());
+
+    // Remove the replication entries too
+    bw = ReplicationTable.getBatchWriter(conn);
+    m = new Mutation(file1);
+    m.put(StatusSection.NAME, tableId1, ProtobufUtil.toValue(newStatus));
+    bw.addMutation(m);
+    bw.flush();
+
+    try {
+      t.join(5000);
+    } catch (InterruptedException e) {
+      Assert.fail("ReplicationOperations.drain did not complete");
+    }
+
+    // New records, but not fully replicated ones don't cause it to complete
+    Assert.assertFalse("Drain somehow finished", done.get());
+    Assert.assertFalse("Saw unexpected exception", exception.get());
+  }
+
+  @Test
+  public void laterCreatedLogsDontBlockExecution() throws Exception {
+    conn.tableOperations().create("foo");
+
+    Text tableId1 = new Text(conn.tableOperations().tableIdMap().get("foo"));
+
+    String file1 = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
+    Status stat = Status.newBuilder().setBegin(0).setEnd(10000).setInfiniteEnd(false).setClosed(false).build();
+
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    Mutation m = new Mutation(file1);
+    StatusSection.add(m, tableId1, ProtobufUtil.toValue(stat));
+    bw.addMutation(m);
+    bw.close();
+
+    bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
+    m.put(ReplicationSection.COLF, tableId1, ProtobufUtil.toValue(stat));
+    bw.addMutation(m);
+
+    bw.close();
+
+    log.info("Reading metadata first time");
+    for (Entry<Key,Value> e : conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      log.info("{}", e.getKey());
+    }
+
+    final AtomicBoolean done = new AtomicBoolean(false);
+    final AtomicBoolean exception = new AtomicBoolean(false);
+    final ReplicationOperationsImpl roi = getReplicationOperations();
+    Thread t = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          roi.drain("foo");
+        } catch (Exception e) {
+          log.error("Got error", e);
+          exception.set(true);
+        }
+        done.set(true);
+      }
+    });
+
+    t.start();
+
+    // We need to wait long enough for the table to read once
+    Thread.sleep(2000);
+
+    // Write another file, but also delete the old files
+    bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+    m = new Mutation(ReplicationSection.getRowPrefix() + "/accumulo/wals/tserver+port/" + UUID.randomUUID());
+    m.put(ReplicationSection.COLF, tableId1, ProtobufUtil.toValue(stat));
+    bw.addMutation(m);
+    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
+    m.putDelete(ReplicationSection.COLF, tableId1);
+    bw.addMutation(m);
+    bw.close();
+
+    log.info("Reading metadata second time");
+    for (Entry<Key,Value> e : conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      log.info("{}", e.getKey());
+    }
+
+    bw = ReplicationTable.getBatchWriter(conn);
+    m = new Mutation(file1);
+    m.putDelete(StatusSection.NAME, tableId1);
+    bw.addMutation(m);
+    bw.close();
+
+    try {
+      t.join(5000);
+    } catch (InterruptedException e) {
+      Assert.fail("ReplicationOperations.drain did not complete");
+    }
+
+    // We should pass immediately because we aren't waiting on both files to be deleted (just the one that we did)
+    Assert.assertTrue("Drain didn't finish", done.get());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/replication/SequentialWorkAssignerIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/SequentialWorkAssignerIT.java b/test/src/main/java/org/apache/accumulo/test/replication/SequentialWorkAssignerIT.java
new file mode 100644
index 0000000..5668a67
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/replication/SequentialWorkAssignerIT.java
@@ -0,0 +1,368 @@
+/*
+ * 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.test.replication;
+
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expectLastCall;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.protobuf.ProtobufUtil;
+import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
+import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
+import org.apache.accumulo.core.replication.ReplicationTable;
+import org.apache.accumulo.core.replication.ReplicationTarget;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.master.replication.SequentialWorkAssigner;
+import org.apache.accumulo.server.replication.DistributedWorkQueueWorkAssignerHelper;
+import org.apache.accumulo.server.replication.proto.Replication.Status;
+import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
+import org.apache.accumulo.server.zookeeper.ZooCache;
+import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class SequentialWorkAssignerIT extends ConfigurableMacBase {
+
+  private Connector conn;
+  private MockSequentialWorkAssigner assigner;
+
+  private static class MockSequentialWorkAssigner extends SequentialWorkAssigner {
+
+    public MockSequentialWorkAssigner(Connector conn) {
+      super(null, conn);
+    }
+
+    @Override
+    public void setConnector(Connector conn) {
+      super.setConnector(conn);
+    }
+
+    @Override
+    public void setQueuedWork(Map<String,Map<String,String>> queuedWork) {
+      super.setQueuedWork(queuedWork);
+    }
+
+    @Override
+    public void setWorkQueue(DistributedWorkQueue workQueue) {
+      super.setWorkQueue(workQueue);
+    }
+
+    @Override
+    public void setMaxQueueSize(int maxQueueSize) {
+      super.setMaxQueueSize(maxQueueSize);
+    }
+
+    @Override
+    public void createWork() {
+      super.createWork();
+    }
+
+    @Override
+    public void setZooCache(ZooCache zooCache) {
+      super.setZooCache(zooCache);
+    }
+
+    @Override
+    public void cleanupFinishedWork() {
+      super.cleanupFinishedWork();
+    }
+
+  }
+
+  @Before
+  public void init() throws Exception {
+    conn = getConnector();
+    assigner = new MockSequentialWorkAssigner(conn);
+    // grant ourselves write to the replication table
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.READ);
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.WRITE);
+    ReplicationTable.setOnline(conn);
+  }
+
+  @Test
+  public void createWorkForFilesInCorrectOrder() throws Exception {
+    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
+    Text serializedTarget = target.toText();
+
+    // Create two mutations, both of which need replication work done
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    // We want the name of file2 to sort before file1
+    String filename1 = "z_file1", filename2 = "a_file1";
+    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
+
+    // File1 was closed before file2, however
+    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(250).build();
+    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(500).build();
+
+    Mutation m = new Mutation(file1);
+    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = new Mutation(file2);
+    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file1, stat1.getCreatedTime());
+    OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file2, stat2.getCreatedTime());
+    OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    bw.close();
+
+    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
+    Map<String,Map<String,String>> queuedWork = new HashMap<>();
+    assigner.setQueuedWork(queuedWork);
+    assigner.setWorkQueue(workQueue);
+    assigner.setMaxQueueSize(Integer.MAX_VALUE);
+
+    // Make sure we expect the invocations in the correct order (accumulo is sorted)
+    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target), file1);
+    expectLastCall().once();
+
+    // file2 is *not* queued because file1 must be replicated first
+
+    replay(workQueue);
+
+    assigner.createWork();
+
+    verify(workQueue);
+
+    Assert.assertEquals(1, queuedWork.size());
+    Assert.assertTrue(queuedWork.containsKey("cluster1"));
+    Map<String,String> cluster1Work = queuedWork.get("cluster1");
+    Assert.assertEquals(1, cluster1Work.size());
+    Assert.assertTrue(cluster1Work.containsKey(target.getSourceTableId()));
+    Assert.assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target), cluster1Work.get(target.getSourceTableId()));
+  }
+
+  @Test
+  public void workAcrossTablesHappensConcurrently() throws Exception {
+    ReplicationTarget target1 = new ReplicationTarget("cluster1", "table1", "1");
+    Text serializedTarget1 = target1.toText();
+
+    ReplicationTarget target2 = new ReplicationTarget("cluster1", "table2", "2");
+    Text serializedTarget2 = target2.toText();
+
+    // Create two mutations, both of which need replication work done
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    // We want the name of file2 to sort before file1
+    String filename1 = "z_file1", filename2 = "a_file1";
+    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
+
+    // File1 was closed before file2, however
+    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(250).build();
+    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(500).build();
+
+    Mutation m = new Mutation(file1);
+    WorkSection.add(m, serializedTarget1, ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = new Mutation(file2);
+    WorkSection.add(m, serializedTarget2, ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file1, stat1.getCreatedTime());
+    OrderSection.add(m, new Text(target1.getSourceTableId()), ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file2, stat2.getCreatedTime());
+    OrderSection.add(m, new Text(target2.getSourceTableId()), ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    bw.close();
+
+    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
+    Map<String,Map<String,String>> queuedWork = new HashMap<>();
+    assigner.setQueuedWork(queuedWork);
+    assigner.setWorkQueue(workQueue);
+    assigner.setMaxQueueSize(Integer.MAX_VALUE);
+
+    // Make sure we expect the invocations in the correct order (accumulo is sorted)
+    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target1), file1);
+    expectLastCall().once();
+
+    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target2), file2);
+    expectLastCall().once();
+
+    // file2 is *not* queued because file1 must be replicated first
+
+    replay(workQueue);
+
+    assigner.createWork();
+
+    verify(workQueue);
+
+    Assert.assertEquals(1, queuedWork.size());
+    Assert.assertTrue(queuedWork.containsKey("cluster1"));
+
+    Map<String,String> cluster1Work = queuedWork.get("cluster1");
+    Assert.assertEquals(2, cluster1Work.size());
+    Assert.assertTrue(cluster1Work.containsKey(target1.getSourceTableId()));
+    Assert.assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target1), cluster1Work.get(target1.getSourceTableId()));
+
+    Assert.assertTrue(cluster1Work.containsKey(target2.getSourceTableId()));
+    Assert.assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target2), cluster1Work.get(target2.getSourceTableId()));
+  }
+
+  @Test
+  public void workAcrossPeersHappensConcurrently() throws Exception {
+    ReplicationTarget target1 = new ReplicationTarget("cluster1", "table1", "1");
+    Text serializedTarget1 = target1.toText();
+
+    ReplicationTarget target2 = new ReplicationTarget("cluster2", "table1", "1");
+    Text serializedTarget2 = target2.toText();
+
+    // Create two mutations, both of which need replication work done
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    // We want the name of file2 to sort before file1
+    String filename1 = "z_file1", filename2 = "a_file1";
+    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
+
+    // File1 was closed before file2, however
+    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(250).build();
+    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(500).build();
+
+    Mutation m = new Mutation(file1);
+    WorkSection.add(m, serializedTarget1, ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = new Mutation(file2);
+    WorkSection.add(m, serializedTarget2, ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file1, stat1.getCreatedTime());
+    OrderSection.add(m, new Text(target1.getSourceTableId()), ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file2, stat2.getCreatedTime());
+    OrderSection.add(m, new Text(target2.getSourceTableId()), ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    bw.close();
+
+    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
+    Map<String,Map<String,String>> queuedWork = new HashMap<>();
+    assigner.setQueuedWork(queuedWork);
+    assigner.setWorkQueue(workQueue);
+    assigner.setMaxQueueSize(Integer.MAX_VALUE);
+
+    // Make sure we expect the invocations in the correct order (accumulo is sorted)
+    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target1), file1);
+    expectLastCall().once();
+
+    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target2), file2);
+    expectLastCall().once();
+
+    // file2 is *not* queued because file1 must be replicated first
+
+    replay(workQueue);
+
+    assigner.createWork();
+
+    verify(workQueue);
+
+    Assert.assertEquals(2, queuedWork.size());
+    Assert.assertTrue(queuedWork.containsKey("cluster1"));
+
+    Map<String,String> cluster1Work = queuedWork.get("cluster1");
+    Assert.assertEquals(1, cluster1Work.size());
+    Assert.assertTrue(cluster1Work.containsKey(target1.getSourceTableId()));
+    Assert.assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target1), cluster1Work.get(target1.getSourceTableId()));
+
+    Map<String,String> cluster2Work = queuedWork.get("cluster2");
+    Assert.assertEquals(1, cluster2Work.size());
+    Assert.assertTrue(cluster2Work.containsKey(target2.getSourceTableId()));
+    Assert.assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target2), cluster2Work.get(target2.getSourceTableId()));
+  }
+
+  @Test
+  public void reprocessingOfCompletedWorkRemovesWork() throws Exception {
+    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
+    Text serializedTarget = target.toText();
+
+    // Create two mutations, both of which need replication work done
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    // We want the name of file2 to sort before file1
+    String filename1 = "z_file1", filename2 = "a_file1";
+    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
+
+    // File1 was closed before file2, however
+    Status stat1 = Status.newBuilder().setBegin(100).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(250).build();
+    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(500).build();
+
+    Mutation m = new Mutation(file1);
+    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = new Mutation(file2);
+    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file1, stat1.getCreatedTime());
+    OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file2, stat2.getCreatedTime());
+    OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    bw.close();
+
+    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
+
+    // Treat filename1 as we have already submitted it for replication
+    Map<String,Map<String,String>> queuedWork = new HashMap<>();
+    Map<String,String> queuedWorkForCluster = new HashMap<>();
+    queuedWorkForCluster.put(target.getSourceTableId(), DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target));
+    queuedWork.put("cluster1", queuedWorkForCluster);
+
+    assigner.setQueuedWork(queuedWork);
+    assigner.setWorkQueue(workQueue);
+    assigner.setMaxQueueSize(Integer.MAX_VALUE);
+
+    // Make sure we expect the invocations in the correct order (accumulo is sorted)
+    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target), file2);
+    expectLastCall().once();
+
+    // file2 is queued because we remove file1 because it's fully replicated
+
+    replay(workQueue);
+
+    assigner.createWork();
+
+    verify(workQueue);
+
+    Assert.assertEquals(1, queuedWork.size());
+    Assert.assertTrue(queuedWork.containsKey("cluster1"));
+    Map<String,String> cluster1Work = queuedWork.get("cluster1");
+    Assert.assertEquals(1, cluster1Work.size());
+    Assert.assertTrue(cluster1Work.containsKey(target.getSourceTableId()));
+    Assert.assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target), cluster1Work.get(target.getSourceTableId()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/replication/StatusMakerIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/StatusMakerIT.java b/test/src/main/java/org/apache/accumulo/test/replication/StatusMakerIT.java
new file mode 100644
index 0000000..cb34ed2
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/replication/StatusMakerIT.java
@@ -0,0 +1,249 @@
+/*
+ * 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.test.replication;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.UUID;
+
+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.Scanner;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
+import org.apache.accumulo.core.protobuf.ProtobufUtil;
+import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
+import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
+import org.apache.accumulo.core.replication.ReplicationTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.master.replication.StatusMaker;
+import org.apache.accumulo.server.replication.StatusUtil;
+import org.apache.accumulo.server.replication.proto.Replication.Status;
+import org.apache.accumulo.server.util.ReplicationTableUtil;
+import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+
+public class StatusMakerIT extends ConfigurableMacBase {
+
+  private Connector conn;
+
+  @Before
+  public void setupInstance() throws Exception {
+    conn = getConnector();
+    ReplicationTable.setOnline(conn);
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.WRITE);
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.READ);
+  }
+
+  @Test
+  public void statusRecordsCreated() throws Exception {
+    String sourceTable = testName.getMethodName();
+    conn.tableOperations().create(sourceTable);
+    ReplicationTableUtil.configureMetadataTable(conn, sourceTable);
+
+    BatchWriter bw = conn.createBatchWriter(sourceTable, new BatchWriterConfig());
+    String walPrefix = "hdfs://localhost:8020/accumulo/wals/tserver+port/";
+    Set<String> files = Sets.newHashSet(walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(),
+        walPrefix + UUID.randomUUID());
+    Map<String,Integer> fileToTableId = new HashMap<>();
+
+    int index = 1;
+    long timeCreated = 0;
+    Map<String,Long> fileToTimeCreated = new HashMap<>();
+    for (String file : files) {
+      Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
+      m.put(ReplicationSection.COLF, new Text(Integer.toString(index)), StatusUtil.fileCreatedValue(timeCreated));
+      fileToTimeCreated.put(file, timeCreated);
+      bw.addMutation(m);
+      fileToTableId.put(file, index);
+      index++;
+      timeCreated++;
+    }
+
+    bw.close();
+
+    StatusMaker statusMaker = new StatusMaker(conn);
+    statusMaker.setSourceTableName(sourceTable);
+
+    statusMaker.run();
+
+    Scanner s = ReplicationTable.getScanner(conn);
+    StatusSection.limit(s);
+    Text file = new Text(), tableId = new Text();
+    for (Entry<Key,Value> entry : s) {
+      StatusSection.getFile(entry.getKey(), file);
+      StatusSection.getTableId(entry.getKey(), tableId);
+
+      Assert.assertTrue("Found unexpected file: " + file, files.contains(file.toString()));
+      Assert.assertEquals(fileToTableId.get(file.toString()), new Integer(tableId.toString()));
+      timeCreated = fileToTimeCreated.get(file.toString());
+      Assert.assertNotNull(timeCreated);
+      Assert.assertEquals(StatusUtil.fileCreated(timeCreated), Status.parseFrom(entry.getValue().get()));
+    }
+  }
+
+  @Test
+  public void openMessagesAreNotDeleted() throws Exception {
+    String sourceTable = testName.getMethodName();
+    conn.tableOperations().create(sourceTable);
+    ReplicationTableUtil.configureMetadataTable(conn, sourceTable);
+
+    BatchWriter bw = conn.createBatchWriter(sourceTable, new BatchWriterConfig());
+    String walPrefix = "hdfs://localhost:8020/accumulo/wals/tserver+port/";
+    Set<String> files = Sets.newHashSet(walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(),
+        walPrefix + UUID.randomUUID());
+    Map<String,Integer> fileToTableId = new HashMap<>();
+
+    int index = 1;
+    long timeCreated = 0;
+    for (String file : files) {
+      Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
+      m.put(ReplicationSection.COLF, new Text(Integer.toString(index)), StatusUtil.fileCreatedValue(timeCreated));
+      bw.addMutation(m);
+      fileToTableId.put(file, index);
+      index++;
+      timeCreated++;
+    }
+
+    bw.close();
+
+    StatusMaker statusMaker = new StatusMaker(conn);
+    statusMaker.setSourceTableName(sourceTable);
+
+    statusMaker.run();
+
+    Scanner s = conn.createScanner(sourceTable, Authorizations.EMPTY);
+    s.setRange(ReplicationSection.getRange());
+    s.fetchColumnFamily(ReplicationSection.COLF);
+    Assert.assertEquals(files.size(), Iterables.size(s));
+  }
+
+  @Test
+  public void closedMessagesAreDeleted() throws Exception {
+    String sourceTable = testName.getMethodName();
+    conn.tableOperations().create(sourceTable);
+    ReplicationTableUtil.configureMetadataTable(conn, sourceTable);
+
+    BatchWriter bw = conn.createBatchWriter(sourceTable, new BatchWriterConfig());
+    String walPrefix = "hdfs://localhost:8020/accumulo/wals/tserver+port/";
+    Set<String> files = Sets.newHashSet(walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(),
+        walPrefix + UUID.randomUUID());
+    Map<String,Integer> fileToTableId = new HashMap<>();
+
+    Status stat = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(true).setCreatedTime(System.currentTimeMillis()).build();
+
+    int index = 1;
+    for (String file : files) {
+      Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
+      m.put(ReplicationSection.COLF, new Text(Integer.toString(index)), ProtobufUtil.toValue(stat));
+      bw.addMutation(m);
+      fileToTableId.put(file, index);
+      index++;
+    }
+
+    bw.close();
+
+    StatusMaker statusMaker = new StatusMaker(conn);
+    statusMaker.setSourceTableName(sourceTable);
+
+    statusMaker.run();
+
+    Scanner s = conn.createScanner(sourceTable, Authorizations.EMPTY);
+    s.setRange(ReplicationSection.getRange());
+    s.fetchColumnFamily(ReplicationSection.COLF);
+    for (Entry<Key,Value> e : s) {
+      System.out.println(e.getKey().toStringNoTruncate() + " " + e.getValue());
+    }
+    s = conn.createScanner(sourceTable, Authorizations.EMPTY);
+    s.setRange(ReplicationSection.getRange());
+    s.fetchColumnFamily(ReplicationSection.COLF);
+    Assert.assertEquals(0, Iterables.size(s));
+
+  }
+
+  @Test
+  public void closedMessagesCreateOrderRecords() throws Exception {
+    String sourceTable = testName.getMethodName();
+    conn.tableOperations().create(sourceTable);
+    ReplicationTableUtil.configureMetadataTable(conn, sourceTable);
+
+    BatchWriter bw = conn.createBatchWriter(sourceTable, new BatchWriterConfig());
+    String walPrefix = "hdfs://localhost:8020/accumulo/wals/tserver+port/";
+    List<String> files = Arrays.asList(walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(),
+        walPrefix + UUID.randomUUID());
+    Map<String,Integer> fileToTableId = new HashMap<>();
+
+    Status.Builder statBuilder = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(true);
+
+    int index = 1;
+    long time = System.currentTimeMillis();
+    for (String file : files) {
+      statBuilder.setCreatedTime(time++);
+      Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
+      m.put(ReplicationSection.COLF, new Text(Integer.toString(index)), ProtobufUtil.toValue(statBuilder.build()));
+      bw.addMutation(m);
+      fileToTableId.put(file, index);
+      index++;
+    }
+
+    bw.close();
+
+    StatusMaker statusMaker = new StatusMaker(conn);
+    statusMaker.setSourceTableName(sourceTable);
+
+    statusMaker.run();
+
+    Scanner s = conn.createScanner(sourceTable, Authorizations.EMPTY);
+    s.setRange(ReplicationSection.getRange());
+    s.fetchColumnFamily(ReplicationSection.COLF);
+    Assert.assertEquals(0, Iterables.size(s));
+
+    s = ReplicationTable.getScanner(conn);
+    OrderSection.limit(s);
+    Iterator<Entry<Key,Value>> iter = s.iterator();
+    Assert.assertTrue("Found no order records in replication table", iter.hasNext());
+
+    Iterator<String> expectedFiles = files.iterator();
+    Text buff = new Text();
+    while (expectedFiles.hasNext() && iter.hasNext()) {
+      String file = expectedFiles.next();
+      Entry<Key,Value> entry = iter.next();
+
+      Assert.assertEquals(file, OrderSection.getFile(entry.getKey(), buff));
+      OrderSection.getTableId(entry.getKey(), buff);
+      Assert.assertEquals(fileToTableId.get(file).intValue(), Integer.parseInt(buff.toString()));
+    }
+
+    Assert.assertFalse("Found more files unexpectedly", expectedFiles.hasNext());
+    Assert.assertFalse("Found more entries in replication table unexpectedly", iter.hasNext());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerIT.java b/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerIT.java
new file mode 100644
index 0000000..048fa94
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/replication/UnorderedWorkAssignerIT.java
@@ -0,0 +1,238 @@
+/*
+ * 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.test.replication;
+
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expectLastCall;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.protobuf.ProtobufUtil;
+import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
+import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
+import org.apache.accumulo.core.replication.ReplicationTable;
+import org.apache.accumulo.core.replication.ReplicationTarget;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.master.replication.UnorderedWorkAssigner;
+import org.apache.accumulo.server.replication.DistributedWorkQueueWorkAssignerHelper;
+import org.apache.accumulo.server.replication.StatusUtil;
+import org.apache.accumulo.server.replication.proto.Replication.Status;
+import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
+import org.apache.accumulo.server.zookeeper.ZooCache;
+import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Test;
+
+public class UnorderedWorkAssignerIT extends ConfigurableMacBase {
+
+  private Connector conn;
+  private MockUnorderedWorkAssigner assigner;
+
+  private static class MockUnorderedWorkAssigner extends UnorderedWorkAssigner {
+    public MockUnorderedWorkAssigner(Connector conn) {
+      super(null, conn);
+    }
+
+    @Override
+    protected void setQueuedWork(Set<String> queuedWork) {
+      super.setQueuedWork(queuedWork);
+    }
+
+    @Override
+    protected void setWorkQueue(DistributedWorkQueue workQueue) {
+      super.setWorkQueue(workQueue);
+    }
+
+    @Override
+    protected boolean queueWork(Path path, ReplicationTarget target) {
+      return super.queueWork(path, target);
+    }
+
+    @Override
+    protected void initializeQueuedWork() {
+      super.initializeQueuedWork();
+    }
+
+    @Override
+    protected Set<String> getQueuedWork() {
+      return super.getQueuedWork();
+    }
+
+    @Override
+    protected void setConnector(Connector conn) {
+      super.setConnector(conn);
+    }
+
+    @Override
+    protected void setMaxQueueSize(int maxQueueSize) {
+      super.setMaxQueueSize(maxQueueSize);
+    }
+
+    @Override
+    protected void createWork() {
+      super.createWork();
+    }
+
+    @Override
+    protected void setZooCache(ZooCache zooCache) {
+      super.setZooCache(zooCache);
+    }
+
+    @Override
+    protected void cleanupFinishedWork() {
+      super.cleanupFinishedWork();
+    }
+  }
+
+  @Before
+  public void init() throws Exception {
+    conn = getConnector();
+    assigner = new MockUnorderedWorkAssigner(conn);
+    ReplicationTable.setOnline(conn);
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.WRITE);
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.READ);
+  }
+
+  @Test
+  public void createWorkForFilesNeedingIt() throws Exception {
+    ReplicationTarget target1 = new ReplicationTarget("cluster1", "table1", "1"), target2 = new ReplicationTarget("cluster1", "table2", "2");
+    Text serializedTarget1 = target1.toText(), serializedTarget2 = target2.toText();
+    String keyTarget1 = target1.getPeerName() + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target1.getRemoteIdentifier()
+        + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target1.getSourceTableId(), keyTarget2 = target2.getPeerName()
+        + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target2.getRemoteIdentifier() + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR
+        + target2.getSourceTableId();
+
+    Status.Builder builder = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(false).setCreatedTime(5l);
+    Status status1 = builder.build();
+    builder.setCreatedTime(10l);
+    Status status2 = builder.build();
+
+    // Create two mutations, both of which need replication work done
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    String filename1 = UUID.randomUUID().toString(), filename2 = UUID.randomUUID().toString();
+    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
+    Mutation m = new Mutation(file1);
+    WorkSection.add(m, serializedTarget1, ProtobufUtil.toValue(status1));
+    bw.addMutation(m);
+    m = OrderSection.createMutation(file1, status1.getCreatedTime());
+    OrderSection.add(m, new Text(target1.getSourceTableId()), ProtobufUtil.toValue(status1));
+    bw.addMutation(m);
+
+    m = new Mutation(file2);
+    WorkSection.add(m, serializedTarget2, ProtobufUtil.toValue(status2));
+    bw.addMutation(m);
+    m = OrderSection.createMutation(file2, status2.getCreatedTime());
+    OrderSection.add(m, new Text(target2.getSourceTableId()), ProtobufUtil.toValue(status2));
+    bw.addMutation(m);
+
+    bw.close();
+
+    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
+    HashSet<String> queuedWork = new HashSet<>();
+    assigner.setQueuedWork(queuedWork);
+    assigner.setWorkQueue(workQueue);
+    assigner.setMaxQueueSize(Integer.MAX_VALUE);
+
+    // Make sure we expect the invocations in the order they were created
+    String key = filename1 + "|" + keyTarget1;
+    workQueue.addWork(key, file1);
+    expectLastCall().once();
+
+    key = filename2 + "|" + keyTarget2;
+    workQueue.addWork(key, file2);
+    expectLastCall().once();
+
+    replay(workQueue);
+
+    assigner.createWork();
+
+    verify(workQueue);
+  }
+
+  @Test
+  public void doNotCreateWorkForFilesNotNeedingIt() throws Exception {
+    ReplicationTarget target1 = new ReplicationTarget("cluster1", "table1", "1"), target2 = new ReplicationTarget("cluster1", "table2", "2");
+    Text serializedTarget1 = target1.toText(), serializedTarget2 = target2.toText();
+
+    // Create two mutations, both of which need replication work done
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    String filename1 = UUID.randomUUID().toString(), filename2 = UUID.randomUUID().toString();
+    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
+
+    Mutation m = new Mutation(file1);
+    WorkSection.add(m, serializedTarget1, StatusUtil.fileCreatedValue(5));
+    bw.addMutation(m);
+
+    m = new Mutation(file2);
+    WorkSection.add(m, serializedTarget2, StatusUtil.fileCreatedValue(10));
+    bw.addMutation(m);
+
+    bw.close();
+
+    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
+    HashSet<String> queuedWork = new HashSet<>();
+    assigner.setQueuedWork(queuedWork);
+    assigner.setMaxQueueSize(Integer.MAX_VALUE);
+
+    replay(workQueue);
+
+    assigner.createWork();
+
+    verify(workQueue);
+  }
+
+  @Test
+  public void workNotReAdded() throws Exception {
+    Set<String> queuedWork = new HashSet<>();
+
+    assigner.setQueuedWork(queuedWork);
+
+    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
+    String serializedTarget = target.getPeerName() + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target.getRemoteIdentifier()
+        + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target.getSourceTableId();
+
+    queuedWork.add("wal1|" + serializedTarget.toString());
+
+    // Create two mutations, both of which need replication work done
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    String file1 = "/accumulo/wal/tserver+port/wal1";
+    Mutation m = new Mutation(file1);
+    WorkSection.add(m, target.toText(), StatusUtil.openWithUnknownLengthValue());
+    bw.addMutation(m);
+
+    bw.close();
+
+    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
+    assigner.setWorkQueue(workQueue);
+    assigner.setMaxQueueSize(Integer.MAX_VALUE);
+
+    replay(workQueue);
+
+    assigner.createWork();
+
+    verify(workQueue);
+  }
+}


[03/14] accumulo git commit: ACCUMULO-3920 Convert more tests from mock

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java
new file mode 100644
index 0000000..bb48761
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloOutputFormatIT.java
@@ -0,0 +1,227 @@
+/*
+ * 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.test.mapred;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientConfiguration;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.MutationsRejectedException;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.mapred.AccumuloInputFormat;
+import org.apache.accumulo.core.client.mapred.AccumuloOutputFormat;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.RecordWriter;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+
+public class AccumuloOutputFormatIT extends ConfigurableMacBase {
+
+  @Override
+  protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "1");
+    cfg.setNumTservers(1);
+  }
+
+  // Prevent regression of ACCUMULO-3709.
+  @Test
+  public void testMapred() throws Exception {
+    Connector connector = getConnector();
+    // create a table and put some data in it
+    connector.tableOperations().create(testName.getMethodName());
+
+    JobConf job = new JobConf();
+    BatchWriterConfig batchConfig = new BatchWriterConfig();
+    // no flushes!!!!!
+    batchConfig.setMaxLatency(0, TimeUnit.MILLISECONDS);
+    // use a single thread to ensure our update session times out
+    batchConfig.setMaxWriteThreads(1);
+    // set the max memory so that we ensure we don't flush on the write.
+    batchConfig.setMaxMemory(Long.MAX_VALUE);
+    AccumuloOutputFormat outputFormat = new AccumuloOutputFormat();
+    AccumuloOutputFormat.setBatchWriterOptions(job, batchConfig);
+    AccumuloOutputFormat.setZooKeeperInstance(job, cluster.getClientConfig());
+    AccumuloOutputFormat.setConnectorInfo(job, "root", new PasswordToken(ROOT_PASSWORD));
+    RecordWriter<Text,Mutation> writer = outputFormat.getRecordWriter(null, job, "Test", null);
+
+    try {
+      for (int i = 0; i < 3; i++) {
+        Mutation m = new Mutation(new Text(String.format("%08d", i)));
+        for (int j = 0; j < 3; j++) {
+          m.put(new Text("cf1"), new Text("cq" + j), new Value((i + "_" + j).getBytes(UTF_8)));
+        }
+        writer.write(new Text(testName.getMethodName()), m);
+      }
+
+    } catch (Exception e) {
+      e.printStackTrace();
+      // we don't want the exception to come from write
+    }
+
+    connector.securityOperations().revokeTablePermission("root", testName.getMethodName(), TablePermission.WRITE);
+
+    try {
+      writer.close(null);
+      fail("Did not throw exception");
+    } catch (IOException ex) {
+      log.info(ex.getMessage(), ex);
+      assertTrue(ex.getCause() instanceof MutationsRejectedException);
+    }
+  }
+
+  private static AssertionError e1 = null;
+
+  private static class MRTester extends Configured implements Tool {
+    private static class TestMapper implements Mapper<Key,Value,Text,Mutation> {
+      Key key = null;
+      int count = 0;
+      OutputCollector<Text,Mutation> finalOutput;
+
+      @Override
+      public void map(Key k, Value v, OutputCollector<Text,Mutation> output, Reporter reporter) throws IOException {
+        finalOutput = output;
+        try {
+          if (key != null)
+            assertEquals(key.getRow().toString(), new String(v.get()));
+          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
+          assertEquals(new String(v.get()), String.format("%09x", count));
+        } catch (AssertionError e) {
+          e1 = e;
+        }
+        key = new Key(k);
+        count++;
+      }
+
+      @Override
+      public void configure(JobConf job) {}
+
+      @Override
+      public void close() throws IOException {
+        Mutation m = new Mutation("total");
+        m.put("", "", Integer.toString(count));
+        finalOutput.collect(new Text(), m);
+      }
+
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+      if (args.length != 6) {
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <inputtable> <outputtable> <instanceName> <zooKeepers>");
+      }
+
+      String user = args[0];
+      String pass = args[1];
+      String table1 = args[2];
+      String table2 = args[3];
+      String instanceName = args[4];
+      String zooKeepers = args[5];
+
+      JobConf job = new JobConf(getConf());
+      job.setJarByClass(this.getClass());
+
+      job.setInputFormat(AccumuloInputFormat.class);
+
+      ClientConfiguration clientConfig = new ClientConfiguration().withInstance(instanceName).withZkHosts(zooKeepers);
+
+      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
+      AccumuloInputFormat.setInputTableName(job, table1);
+      AccumuloInputFormat.setZooKeeperInstance(job, clientConfig);
+
+      job.setMapperClass(TestMapper.class);
+      job.setMapOutputKeyClass(Key.class);
+      job.setMapOutputValueClass(Value.class);
+      job.setOutputFormat(AccumuloOutputFormat.class);
+      job.setOutputKeyClass(Text.class);
+      job.setOutputValueClass(Mutation.class);
+
+      AccumuloOutputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
+      AccumuloOutputFormat.setCreateTables(job, false);
+      AccumuloOutputFormat.setDefaultTableName(job, table2);
+      AccumuloOutputFormat.setZooKeeperInstance(job, clientConfig);
+
+      job.setNumReduceTasks(0);
+
+      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
+    }
+
+    public static void main(String[] args) throws Exception {
+      Configuration conf = new Configuration();
+      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
+      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
+    }
+  }
+
+  @Test
+  public void testMR() throws Exception {
+    Connector c = getConnector();
+    String instanceName = getCluster().getInstanceName();
+    String table1 = instanceName + "_t1";
+    String table2 = instanceName + "_t2";
+    c.tableOperations().create(table1);
+    c.tableOperations().create(table2);
+    BatchWriter bw = c.createBatchWriter(table1, new BatchWriterConfig());
+    for (int i = 0; i < 100; i++) {
+      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
+      bw.addMutation(m);
+    }
+    bw.close();
+
+    MRTester.main(new String[] {"root", ROOT_PASSWORD, table1, table2, instanceName, getCluster().getZooKeepers()});
+    assertNull(e1);
+
+    Scanner scanner = c.createScanner(table2, new Authorizations());
+    Iterator<Entry<Key,Value>> iter = scanner.iterator();
+    assertTrue(iter.hasNext());
+    Entry<Key,Value> entry = iter.next();
+    assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
+    assertFalse(iter.hasNext());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloRowInputFormatIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloRowInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloRowInputFormatIT.java
new file mode 100644
index 0000000..d266d3d
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloRowInputFormatIT.java
@@ -0,0 +1,202 @@
+/*
+ * 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.test.mapred;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+
+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.MutationsRejectedException;
+import org.apache.accumulo.core.client.mapred.AccumuloInputFormat;
+import org.apache.accumulo.core.client.mapred.AccumuloRowInputFormat;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.KeyValue;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.util.PeekingIterator;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.lib.NullOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class AccumuloRowInputFormatIT extends AccumuloClusterHarness {
+
+  private static final String ROW1 = "row1";
+  private static final String ROW2 = "row2";
+  private static final String ROW3 = "row3";
+  private static final String COLF1 = "colf1";
+  private static List<Entry<Key,Value>> row1;
+  private static List<Entry<Key,Value>> row2;
+  private static List<Entry<Key,Value>> row3;
+  private static AssertionError e1 = null;
+  private static AssertionError e2 = null;
+
+  @BeforeClass
+  public static void prepareRows() {
+    row1 = new ArrayList<Entry<Key,Value>>();
+    row1.add(new KeyValue(new Key(ROW1, COLF1, "colq1"), "v1".getBytes()));
+    row1.add(new KeyValue(new Key(ROW1, COLF1, "colq2"), "v2".getBytes()));
+    row1.add(new KeyValue(new Key(ROW1, "colf2", "colq3"), "v3".getBytes()));
+    row2 = new ArrayList<Entry<Key,Value>>();
+    row2.add(new KeyValue(new Key(ROW2, COLF1, "colq4"), "v4".getBytes()));
+    row3 = new ArrayList<Entry<Key,Value>>();
+    row3.add(new KeyValue(new Key(ROW3, COLF1, "colq5"), "v5".getBytes()));
+  }
+
+  private static void checkLists(final List<Entry<Key,Value>> first, final Iterator<Entry<Key,Value>> second) {
+    int entryIndex = 0;
+    while (second.hasNext()) {
+      final Entry<Key,Value> entry = second.next();
+      assertEquals("Keys should be equal", first.get(entryIndex).getKey(), entry.getKey());
+      assertEquals("Values should be equal", first.get(entryIndex).getValue(), entry.getValue());
+      entryIndex++;
+    }
+  }
+
+  private static void insertList(final BatchWriter writer, final List<Entry<Key,Value>> list) throws MutationsRejectedException {
+    for (Entry<Key,Value> e : list) {
+      final Key key = e.getKey();
+      final Mutation mutation = new Mutation(key.getRow());
+      ColumnVisibility colVisibility = new ColumnVisibility(key.getColumnVisibility());
+      mutation.put(key.getColumnFamily(), key.getColumnQualifier(), colVisibility, key.getTimestamp(), e.getValue());
+      writer.addMutation(mutation);
+    }
+  }
+
+  private static class MRTester extends Configured implements Tool {
+    public static class TestMapper implements Mapper<Text,PeekingIterator<Entry<Key,Value>>,Key,Value> {
+      int count = 0;
+
+      @Override
+      public void map(Text k, PeekingIterator<Entry<Key,Value>> v, OutputCollector<Key,Value> output, Reporter reporter) throws IOException {
+        try {
+          switch (count) {
+            case 0:
+              assertEquals("Current key should be " + ROW1, new Text(ROW1), k);
+              checkLists(row1, v);
+              break;
+            case 1:
+              assertEquals("Current key should be " + ROW2, new Text(ROW2), k);
+              checkLists(row2, v);
+              break;
+            case 2:
+              assertEquals("Current key should be " + ROW3, new Text(ROW3), k);
+              checkLists(row3, v);
+              break;
+            default:
+              assertTrue(false);
+          }
+        } catch (AssertionError e) {
+          e1 = e;
+        }
+        count++;
+      }
+
+      @Override
+      public void configure(JobConf job) {}
+
+      @Override
+      public void close() throws IOException {
+        try {
+          assertEquals(3, count);
+        } catch (AssertionError e) {
+          e2 = e;
+        }
+      }
+
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+      if (args.length != 1) {
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <table>");
+      }
+
+      String user = getAdminPrincipal();
+      AuthenticationToken pass = getAdminToken();
+      String table = args[0];
+
+      JobConf job = new JobConf(getConf());
+      job.setJarByClass(this.getClass());
+
+      job.setInputFormat(AccumuloRowInputFormat.class);
+
+      AccumuloInputFormat.setConnectorInfo(job, user, pass);
+      AccumuloInputFormat.setInputTableName(job, table);
+      AccumuloRowInputFormat.setZooKeeperInstance(job, getCluster().getClientConfig());
+
+      job.setMapperClass(TestMapper.class);
+      job.setMapOutputKeyClass(Key.class);
+      job.setMapOutputValueClass(Value.class);
+      job.setOutputFormat(NullOutputFormat.class);
+
+      job.setNumReduceTasks(0);
+
+      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
+    }
+
+    public static void main(String[] args) throws Exception {
+      Configuration conf = new Configuration();
+      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
+      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
+    }
+  }
+
+  @Test
+  public void test() throws Exception {
+    final Connector conn = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    conn.tableOperations().create(tableName);
+    BatchWriter writer = null;
+    try {
+      writer = conn.createBatchWriter(tableName, new BatchWriterConfig());
+      insertList(writer, row1);
+      insertList(writer, row2);
+      insertList(writer, row3);
+    } finally {
+      if (writer != null) {
+        writer.close();
+      }
+    }
+    MRTester.main(new String[] {tableName});
+    assertNull(e1);
+    assertNull(e2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java
new file mode 100644
index 0000000..e30a033
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/mapred/TokenFileIT.java
@@ -0,0 +1,174 @@
+/*
+ * 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.test.mapred;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.Iterator;
+import java.util.Map.Entry;
+
+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.Scanner;
+import org.apache.accumulo.core.client.impl.Credentials;
+import org.apache.accumulo.core.client.mapred.AccumuloInputFormat;
+import org.apache.accumulo.core.client.mapred.AccumuloOutputFormat;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TokenFileIT extends AccumuloClusterHarness {
+  private static AssertionError e1 = null;
+
+  private static class MRTokenFileTester extends Configured implements Tool {
+    private static class TestMapper implements Mapper<Key,Value,Text,Mutation> {
+      Key key = null;
+      int count = 0;
+      OutputCollector<Text,Mutation> finalOutput;
+
+      @Override
+      public void map(Key k, Value v, OutputCollector<Text,Mutation> output, Reporter reporter) throws IOException {
+        finalOutput = output;
+        try {
+          if (key != null)
+            assertEquals(key.getRow().toString(), new String(v.get()));
+          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
+          assertEquals(new String(v.get()), String.format("%09x", count));
+        } catch (AssertionError e) {
+          e1 = e;
+        }
+        key = new Key(k);
+        count++;
+      }
+
+      @Override
+      public void configure(JobConf job) {}
+
+      @Override
+      public void close() throws IOException {
+        Mutation m = new Mutation("total");
+        m.put("", "", Integer.toString(count));
+        finalOutput.collect(new Text(), m);
+      }
+
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+      if (args.length != 3) {
+        throw new IllegalArgumentException("Usage : " + MRTokenFileTester.class.getName() + " <token file> <inputtable> <outputtable>");
+      }
+
+      String user = getAdminPrincipal();
+      String tokenFile = args[0];
+      String table1 = args[1];
+      String table2 = args[2];
+
+      JobConf job = new JobConf(getConf());
+      job.setJarByClass(this.getClass());
+
+      job.setInputFormat(AccumuloInputFormat.class);
+
+      AccumuloInputFormat.setConnectorInfo(job, user, tokenFile);
+      AccumuloInputFormat.setInputTableName(job, table1);
+      AccumuloInputFormat.setZooKeeperInstance(job, getCluster().getClientConfig());
+
+      job.setMapperClass(TestMapper.class);
+      job.setMapOutputKeyClass(Key.class);
+      job.setMapOutputValueClass(Value.class);
+      job.setOutputFormat(AccumuloOutputFormat.class);
+      job.setOutputKeyClass(Text.class);
+      job.setOutputValueClass(Mutation.class);
+
+      AccumuloOutputFormat.setConnectorInfo(job, user, tokenFile);
+      AccumuloOutputFormat.setCreateTables(job, false);
+      AccumuloOutputFormat.setDefaultTableName(job, table2);
+      AccumuloOutputFormat.setZooKeeperInstance(job, getCluster().getClientConfig());
+
+      job.setNumReduceTasks(0);
+
+      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
+    }
+
+    public static void main(String[] args) throws Exception {
+      Configuration conf = CachedConfiguration.getInstance();
+      conf.set("hadoop.tmp.dir", new File(args[0]).getParent());
+      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
+      assertEquals(0, ToolRunner.run(conf, new MRTokenFileTester(), args));
+    }
+  }
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
+
+  @Test
+  public void testMR() throws Exception {
+    String[] tableNames = getUniqueNames(2);
+    String table1 = tableNames[0];
+    String table2 = tableNames[1];
+    Connector c = getConnector();
+    c.tableOperations().create(table1);
+    c.tableOperations().create(table2);
+    BatchWriter bw = c.createBatchWriter(table1, new BatchWriterConfig());
+    for (int i = 0; i < 100; i++) {
+      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
+      bw.addMutation(m);
+    }
+    bw.close();
+
+    File tf = folder.newFile("root_test.pw");
+    PrintStream out = new PrintStream(tf);
+    String outString = new Credentials(getAdminPrincipal(), getAdminToken()).serialize();
+    out.println(outString);
+    out.close();
+
+    MRTokenFileTester.main(new String[] {tf.getAbsolutePath(), table1, table2});
+    assertNull(e1);
+
+    Scanner scanner = c.createScanner(table2, new Authorizations());
+    Iterator<Entry<Key,Value>> iter = scanner.iterator();
+    assertTrue(iter.hasNext());
+    Entry<Key,Value> entry = iter.next();
+    assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
+    assertFalse(iter.hasNext());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java
new file mode 100644
index 0000000..8f53378
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java
@@ -0,0 +1,211 @@
+/*
+ * 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.test.mapreduce;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+
+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.mapreduce.AccumuloFileOutputFormat;
+import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+
+public class AccumuloFileOutputFormatIT extends AccumuloClusterHarness {
+
+  private String PREFIX;
+  private String BAD_TABLE;
+  private String TEST_TABLE;
+  private String EMPTY_TABLE;
+
+  @Override
+  protected int defaultTimeoutSeconds() {
+    return 4 * 60;
+  }
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
+
+  @Before
+  public void setup() throws Exception {
+    PREFIX = testName.getMethodName() + "_";
+    BAD_TABLE = PREFIX + "_mapreduce_bad_table";
+    TEST_TABLE = PREFIX + "_mapreduce_test_table";
+    EMPTY_TABLE = PREFIX + "_mapreduce_empty_table";
+
+    Connector c = getConnector();
+    c.tableOperations().create(EMPTY_TABLE);
+    c.tableOperations().create(TEST_TABLE);
+    c.tableOperations().create(BAD_TABLE);
+    BatchWriter bw = c.createBatchWriter(TEST_TABLE, new BatchWriterConfig());
+    Mutation m = new Mutation("Key");
+    m.put("", "", "");
+    bw.addMutation(m);
+    bw.close();
+    bw = c.createBatchWriter(BAD_TABLE, new BatchWriterConfig());
+    m = new Mutation("r1");
+    m.put("cf1", "cq1", "A&B");
+    m.put("cf1", "cq1", "A&B");
+    m.put("cf1", "cq2", "A&");
+    bw.addMutation(m);
+    bw.close();
+  }
+
+  @Test
+  public void testEmptyWrite() throws Exception {
+    handleWriteTests(false);
+  }
+
+  @Test
+  public void testRealWrite() throws Exception {
+    handleWriteTests(true);
+  }
+
+  private static class MRTester extends Configured implements Tool {
+    private static class BadKeyMapper extends Mapper<Key,Value,Key,Value> {
+      int index = 0;
+
+      @Override
+      protected void map(Key key, Value value, Context context) throws IOException, InterruptedException {
+        String table = context.getConfiguration().get("MRTester_tableName");
+        assertNotNull(table);
+        try {
+          try {
+            context.write(key, value);
+            if (index == 2)
+              assertTrue(false);
+          } catch (Exception e) {
+            assertEquals(2, index);
+          }
+        } catch (AssertionError e) {
+          assertionErrors.put(table + "_map", e);
+        }
+        index++;
+      }
+
+      @Override
+      protected void cleanup(Context context) throws IOException, InterruptedException {
+        String table = context.getConfiguration().get("MRTester_tableName");
+        assertNotNull(table);
+        try {
+          assertEquals(2, index);
+        } catch (AssertionError e) {
+          assertionErrors.put(table + "_cleanup", e);
+        }
+      }
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+      if (args.length != 2) {
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <table> <outputfile>");
+      }
+
+      String table = args[0];
+      assertionErrors.put(table + "_map", new AssertionError("Dummy_map"));
+      assertionErrors.put(table + "_cleanup", new AssertionError("Dummy_cleanup"));
+
+      Job job = Job.getInstance(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
+      job.setJarByClass(this.getClass());
+
+      job.setInputFormatClass(AccumuloInputFormat.class);
+
+      AccumuloInputFormat.setConnectorInfo(job, getAdminPrincipal(), getAdminToken());
+      AccumuloInputFormat.setInputTableName(job, table);
+      AccumuloInputFormat.setZooKeeperInstance(job, getCluster().getClientConfig());
+      AccumuloFileOutputFormat.setOutputPath(job, new Path(args[1]));
+
+      job.setMapperClass(table.endsWith("_mapreduce_bad_table") ? BadKeyMapper.class : Mapper.class);
+      job.setMapOutputKeyClass(Key.class);
+      job.setMapOutputValueClass(Value.class);
+      job.setOutputFormatClass(AccumuloFileOutputFormat.class);
+      job.getConfiguration().set("MRTester_tableName", table);
+
+      job.setNumReduceTasks(0);
+
+      job.waitForCompletion(true);
+
+      return job.isSuccessful() ? 0 : 1;
+    }
+
+    public static void main(String[] args) throws Exception {
+      Configuration conf = new Configuration();
+      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
+      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
+    }
+  }
+
+  private void handleWriteTests(boolean content) throws Exception {
+    File f = folder.newFile(testName.getMethodName());
+    assertTrue(f.delete());
+    MRTester.main(new String[] {content ? TEST_TABLE : EMPTY_TABLE, f.getAbsolutePath()});
+
+    assertTrue(f.exists());
+    File[] files = f.listFiles(new FileFilter() {
+      @Override
+      public boolean accept(File file) {
+        return file.getName().startsWith("part-m-");
+      }
+    });
+    assertNotNull(files);
+    if (content) {
+      assertEquals(1, files.length);
+      assertTrue(files[0].exists());
+    } else {
+      assertEquals(0, files.length);
+    }
+  }
+
+  // track errors in the map reduce job; jobs insert a dummy error for the map and cleanup tasks (to ensure test correctness),
+  // so error tests should check to see if there is at least one error (could be more depending on the test) rather than zero
+  private static Multimap<String,AssertionError> assertionErrors = ArrayListMultimap.create();
+
+  @Test
+  public void writeBadVisibility() throws Exception {
+    File f = folder.newFile(testName.getMethodName());
+    assertTrue(f.delete());
+    MRTester.main(new String[] {BAD_TABLE, f.getAbsolutePath()});
+    assertTrue(f.exists());
+    assertEquals(1, assertionErrors.get(BAD_TABLE + "_map").size());
+    assertEquals(1, assertionErrors.get(BAD_TABLE + "_cleanup").size());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloInputFormatIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloInputFormatIT.java
new file mode 100644
index 0000000..1ca4f92
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloInputFormatIT.java
@@ -0,0 +1,479 @@
+/*
+ * 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.test.mapreduce;
+
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+import static java.lang.System.currentTimeMillis;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientConfiguration;
+import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
+import org.apache.accumulo.core.client.mapreduce.RangeInputSplit;
+import org.apache.accumulo.core.client.mapreduce.impl.BatchInputSplit;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.conf.Property;
+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.security.Authorizations;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Level;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+
+public class AccumuloInputFormatIT extends AccumuloClusterHarness {
+
+  AccumuloInputFormat inputFormat;
+
+  @Override
+  protected int defaultTimeoutSeconds() {
+    return 4 * 60;
+  }
+
+  @Override
+  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setNumTservers(1);
+  }
+
+  @Before
+  public void before() {
+    inputFormat = new AccumuloInputFormat();
+  }
+
+  /**
+   * Tests several different paths through the getSplits() method by setting different properties and verifying the results.
+   */
+  @Test
+  public void testGetSplits() throws Exception {
+    Connector conn = getConnector();
+    String table = getUniqueNames(1)[0];
+    conn.tableOperations().create(table);
+    insertData(table, currentTimeMillis());
+
+    ClientConfiguration clientConf = cluster.getClientConfig();
+    AccumuloConfiguration clusterClientConf = new ConfigurationCopy(new DefaultConfiguration());
+
+    // Pass SSL and CredentialProvider options into the ClientConfiguration given to AccumuloInputFormat
+    boolean sslEnabled = Boolean.valueOf(clusterClientConf.get(Property.INSTANCE_RPC_SSL_ENABLED));
+    if (sslEnabled) {
+      ClientProperty[] sslProperties = new ClientProperty[] {ClientProperty.INSTANCE_RPC_SSL_ENABLED, ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH,
+          ClientProperty.RPC_SSL_KEYSTORE_PATH, ClientProperty.RPC_SSL_KEYSTORE_TYPE, ClientProperty.RPC_SSL_KEYSTORE_PASSWORD,
+          ClientProperty.RPC_SSL_TRUSTSTORE_PATH, ClientProperty.RPC_SSL_TRUSTSTORE_TYPE, ClientProperty.RPC_SSL_TRUSTSTORE_PASSWORD,
+          ClientProperty.RPC_USE_JSSE, ClientProperty.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS};
+
+      for (ClientProperty prop : sslProperties) {
+        // The default property is returned if it's not in the ClientConfiguration so we don't have to check if the value is actually defined
+        clientConf.setProperty(prop, clusterClientConf.get(prop.getKey()));
+      }
+    }
+
+    Job job = Job.getInstance();
+    AccumuloInputFormat.setInputTableName(job, table);
+    AccumuloInputFormat.setZooKeeperInstance(job, clientConf);
+    AccumuloInputFormat.setConnectorInfo(job, getAdminPrincipal(), getAdminToken());
+
+    // split table
+    TreeSet<Text> splitsToAdd = new TreeSet<Text>();
+    for (int i = 0; i < 10000; i += 1000)
+      splitsToAdd.add(new Text(String.format("%09d", i)));
+    conn.tableOperations().addSplits(table, splitsToAdd);
+    sleepUninterruptibly(500, TimeUnit.MILLISECONDS); // wait for splits to be propagated
+
+    // get splits without setting any range
+    Collection<Text> actualSplits = conn.tableOperations().listSplits(table);
+    List<InputSplit> splits = inputFormat.getSplits(job);
+    assertEquals(actualSplits.size() + 1, splits.size()); // No ranges set on the job so it'll start with -inf
+
+    // set ranges and get splits
+    List<Range> ranges = new ArrayList<Range>();
+    for (Text text : actualSplits)
+      ranges.add(new Range(text));
+    AccumuloInputFormat.setRanges(job, ranges);
+    splits = inputFormat.getSplits(job);
+    assertEquals(actualSplits.size(), splits.size());
+
+    // offline mode
+    AccumuloInputFormat.setOfflineTableScan(job, true);
+    try {
+      inputFormat.getSplits(job);
+      fail("An exception should have been thrown");
+    } catch (IOException e) {}
+
+    conn.tableOperations().offline(table, true);
+    splits = inputFormat.getSplits(job);
+    assertEquals(actualSplits.size(), splits.size());
+
+    // auto adjust ranges
+    ranges = new ArrayList<Range>();
+    for (int i = 0; i < 5; i++)
+      // overlapping ranges
+      ranges.add(new Range(String.format("%09d", i), String.format("%09d", i + 2)));
+    AccumuloInputFormat.setRanges(job, ranges);
+    splits = inputFormat.getSplits(job);
+    assertEquals(2, splits.size());
+
+    AccumuloInputFormat.setAutoAdjustRanges(job, false);
+    splits = inputFormat.getSplits(job);
+    assertEquals(ranges.size(), splits.size());
+
+    // BatchScan not available for offline scans
+    AccumuloInputFormat.setBatchScan(job, true);
+    // Reset auto-adjust ranges too
+    AccumuloInputFormat.setAutoAdjustRanges(job, true);
+
+    AccumuloInputFormat.setOfflineTableScan(job, true);
+    try {
+      inputFormat.getSplits(job);
+      fail("An exception should have been thrown");
+    } catch (IllegalArgumentException e) {}
+
+    conn.tableOperations().online(table, true);
+    AccumuloInputFormat.setOfflineTableScan(job, false);
+
+    // test for resumption of success
+    splits = inputFormat.getSplits(job);
+    assertEquals(2, splits.size());
+
+    // BatchScan not available with isolated iterators
+    AccumuloInputFormat.setScanIsolation(job, true);
+    try {
+      inputFormat.getSplits(job);
+      fail("An exception should have been thrown");
+    } catch (IllegalArgumentException e) {}
+    AccumuloInputFormat.setScanIsolation(job, false);
+
+    // test for resumption of success
+    splits = inputFormat.getSplits(job);
+    assertEquals(2, splits.size());
+
+    // BatchScan not available with local iterators
+    AccumuloInputFormat.setLocalIterators(job, true);
+    try {
+      inputFormat.getSplits(job);
+      fail("An exception should have been thrown");
+    } catch (IllegalArgumentException e) {}
+    AccumuloInputFormat.setLocalIterators(job, false);
+
+    // Check we are getting back correct type pf split
+    conn.tableOperations().online(table);
+    splits = inputFormat.getSplits(job);
+    for (InputSplit split : splits)
+      assert (split instanceof BatchInputSplit);
+
+    // We should divide along the tablet lines similar to when using `setAutoAdjustRanges(job, true)`
+    assertEquals(2, splits.size());
+  }
+
+  private void insertData(String tableName, long ts) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    BatchWriter bw = getConnector().createBatchWriter(tableName, null);
+
+    for (int i = 0; i < 10000; i++) {
+      String row = String.format("%09d", i);
+
+      Mutation m = new Mutation(new Text(row));
+      m.put(new Text("cf1"), new Text("cq1"), ts, new Value(("" + i).getBytes()));
+      bw.addMutation(m);
+    }
+    bw.close();
+  }
+
+  // track errors in the map reduce job; jobs insert a dummy error for the map and cleanup tasks (to ensure test correctness),
+  // so error tests should check to see if there is at least one error (could be more depending on the test) rather than zero
+  private static Multimap<String,AssertionError> assertionErrors = ArrayListMultimap.create();
+
+  private static class MRTester extends Configured implements Tool {
+    private static class TestMapper extends Mapper<Key,Value,Key,Value> {
+      Key key = null;
+      int count = 0;
+
+      @Override
+      protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
+        String table = context.getConfiguration().get("MRTester_tableName");
+        assertNotNull(table);
+        try {
+          if (key != null)
+            assertEquals(key.getRow().toString(), new String(v.get()));
+          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
+          assertEquals(new String(v.get()), String.format("%09x", count));
+        } catch (AssertionError e) {
+          assertionErrors.put(table + "_map", e);
+        }
+        key = new Key(k);
+        count++;
+      }
+
+      @Override
+      protected void cleanup(Context context) throws IOException, InterruptedException {
+        String table = context.getConfiguration().get("MRTester_tableName");
+        assertNotNull(table);
+        try {
+          assertEquals(100, count);
+        } catch (AssertionError e) {
+          assertionErrors.put(table + "_cleanup", e);
+        }
+      }
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+      if (args.length != 2 && args.length != 3) {
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <table> <inputFormatClass> [<batchScan>]");
+      }
+
+      String table = args[0];
+      String inputFormatClassName = args[1];
+      Boolean batchScan = false;
+      if (args.length == 3)
+        batchScan = Boolean.parseBoolean(args[2]);
+
+      assertionErrors.put(table + "_map", new AssertionError("Dummy_map"));
+      assertionErrors.put(table + "_cleanup", new AssertionError("Dummy_cleanup"));
+
+      @SuppressWarnings("unchecked")
+      Class<? extends InputFormat<?,?>> inputFormatClass = (Class<? extends InputFormat<?,?>>) Class.forName(inputFormatClassName);
+
+      Job job = Job.getInstance(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
+      job.setJarByClass(this.getClass());
+      job.getConfiguration().set("MRTester_tableName", table);
+
+      job.setInputFormatClass(inputFormatClass);
+
+      AccumuloInputFormat.setZooKeeperInstance(job, cluster.getClientConfig());
+      AccumuloInputFormat.setConnectorInfo(job, getAdminPrincipal(), getAdminToken());
+      AccumuloInputFormat.setInputTableName(job, table);
+      AccumuloInputFormat.setBatchScan(job, batchScan);
+
+      job.setMapperClass(TestMapper.class);
+      job.setMapOutputKeyClass(Key.class);
+      job.setMapOutputValueClass(Value.class);
+      job.setOutputFormatClass(NullOutputFormat.class);
+
+      job.setNumReduceTasks(0);
+
+      job.waitForCompletion(true);
+
+      return job.isSuccessful() ? 0 : 1;
+    }
+
+    public static int main(String[] args) throws Exception {
+      Configuration conf = new Configuration();
+      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
+      return ToolRunner.run(conf, new MRTester(), args);
+    }
+  }
+
+  @Test
+  public void testMap() throws Exception {
+    final String TEST_TABLE_1 = getUniqueNames(1)[0];
+
+    Connector c = getConnector();
+    c.tableOperations().create(TEST_TABLE_1);
+    BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
+    for (int i = 0; i < 100; i++) {
+      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
+      bw.addMutation(m);
+    }
+    bw.close();
+
+    Assert.assertEquals(0, MRTester.main(new String[] {TEST_TABLE_1, AccumuloInputFormat.class.getName()}));
+    assertEquals(1, assertionErrors.get(TEST_TABLE_1 + "_map").size());
+    assertEquals(1, assertionErrors.get(TEST_TABLE_1 + "_cleanup").size());
+  }
+
+  @Test
+  public void testMapWithBatchScanner() throws Exception {
+    final String TEST_TABLE_2 = getUniqueNames(1)[0];
+
+    Connector c = getConnector();
+    c.tableOperations().create(TEST_TABLE_2);
+    BatchWriter bw = c.createBatchWriter(TEST_TABLE_2, new BatchWriterConfig());
+    for (int i = 0; i < 100; i++) {
+      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
+      bw.addMutation(m);
+    }
+    bw.close();
+
+    Assert.assertEquals(0, MRTester.main(new String[] {TEST_TABLE_2, AccumuloInputFormat.class.getName(), "True"}));
+    assertEquals(1, assertionErrors.get(TEST_TABLE_2 + "_map").size());
+    assertEquals(1, assertionErrors.get(TEST_TABLE_2 + "_cleanup").size());
+  }
+
+  @Test
+  public void testCorrectRangeInputSplits() throws Exception {
+    Job job = Job.getInstance();
+
+    String table = getUniqueNames(1)[0];
+    Authorizations auths = new Authorizations("foo");
+    Collection<Pair<Text,Text>> fetchColumns = Collections.singleton(new Pair<Text,Text>(new Text("foo"), new Text("bar")));
+    boolean isolated = true, localIters = true;
+    Level level = Level.WARN;
+
+    Connector connector = getConnector();
+    connector.tableOperations().create(table);
+
+    AccumuloInputFormat.setZooKeeperInstance(job, cluster.getClientConfig());
+    AccumuloInputFormat.setConnectorInfo(job, getAdminPrincipal(), getAdminToken());
+
+    AccumuloInputFormat.setInputTableName(job, table);
+    AccumuloInputFormat.setScanAuthorizations(job, auths);
+    AccumuloInputFormat.setScanIsolation(job, isolated);
+    AccumuloInputFormat.setLocalIterators(job, localIters);
+    AccumuloInputFormat.fetchColumns(job, fetchColumns);
+    AccumuloInputFormat.setLogLevel(job, level);
+
+    AccumuloInputFormat aif = new AccumuloInputFormat();
+
+    List<InputSplit> splits = aif.getSplits(job);
+
+    Assert.assertEquals(1, splits.size());
+
+    InputSplit split = splits.get(0);
+
+    Assert.assertEquals(RangeInputSplit.class, split.getClass());
+
+    RangeInputSplit risplit = (RangeInputSplit) split;
+
+    Assert.assertEquals(getAdminPrincipal(), risplit.getPrincipal());
+    Assert.assertEquals(table, risplit.getTableName());
+    Assert.assertEquals(getAdminToken(), risplit.getToken());
+    Assert.assertEquals(auths, risplit.getAuths());
+    Assert.assertEquals(getConnector().getInstance().getInstanceName(), risplit.getInstanceName());
+    Assert.assertEquals(isolated, risplit.isIsolatedScan());
+    Assert.assertEquals(localIters, risplit.usesLocalIterators());
+    Assert.assertEquals(fetchColumns, risplit.getFetchedColumns());
+    Assert.assertEquals(level, risplit.getLogLevel());
+  }
+
+  @Test
+  public void testPartialInputSplitDelegationToConfiguration() throws Exception {
+    String table = getUniqueNames(1)[0];
+    Connector c = getConnector();
+    c.tableOperations().create(table);
+    BatchWriter bw = c.createBatchWriter(table, new BatchWriterConfig());
+    for (int i = 0; i < 100; i++) {
+      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
+      bw.addMutation(m);
+    }
+    bw.close();
+
+    Assert.assertEquals(0, MRTester.main(new String[] {table, EmptySplitsAccumuloInputFormat.class.getName()}));
+    assertEquals(1, assertionErrors.get(table + "_map").size());
+    assertEquals(1, assertionErrors.get(table + "_cleanup").size());
+  }
+
+  @Test
+  public void testPartialFailedInputSplitDelegationToConfiguration() throws Exception {
+    String table = getUniqueNames(1)[0];
+    Connector c = getConnector();
+    c.tableOperations().create(table);
+    BatchWriter bw = c.createBatchWriter(table, new BatchWriterConfig());
+    for (int i = 0; i < 100; i++) {
+      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
+      bw.addMutation(m);
+    }
+    bw.close();
+
+    Assert.assertEquals(1, MRTester.main(new String[] {table, BadPasswordSplitsAccumuloInputFormat.class.getName()}));
+    assertEquals(1, assertionErrors.get(table + "_map").size());
+    // We should fail when the RecordReader fails to get the next key/value pair, because the record reader is set up with a clientcontext, rather than a
+    // connector, so it doesn't do fast-fail on bad credentials
+    assertEquals(2, assertionErrors.get(table + "_cleanup").size());
+  }
+
+  /**
+   * AccumuloInputFormat which returns an "empty" RangeInputSplit
+   */
+  public static class BadPasswordSplitsAccumuloInputFormat extends AccumuloInputFormat {
+
+    @Override
+    public List<InputSplit> getSplits(JobContext context) throws IOException {
+      List<InputSplit> splits = super.getSplits(context);
+
+      for (InputSplit split : splits) {
+        org.apache.accumulo.core.client.mapreduce.RangeInputSplit rangeSplit = (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) split;
+        rangeSplit.setToken(new PasswordToken("anythingelse"));
+      }
+
+      return splits;
+    }
+  }
+
+  /**
+   * AccumuloInputFormat which returns an "empty" RangeInputSplit
+   */
+  public static class EmptySplitsAccumuloInputFormat extends AccumuloInputFormat {
+
+    @Override
+    public List<InputSplit> getSplits(JobContext context) throws IOException {
+      List<InputSplit> oldSplits = super.getSplits(context);
+      List<InputSplit> newSplits = new ArrayList<InputSplit>(oldSplits.size());
+
+      // Copy only the necessary information
+      for (InputSplit oldSplit : oldSplits) {
+        org.apache.accumulo.core.client.mapreduce.RangeInputSplit newSplit = new org.apache.accumulo.core.client.mapreduce.RangeInputSplit(
+            (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) oldSplit);
+        newSplits.add(newSplit);
+      }
+
+      return newSplits;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloMultiTableInputFormatIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloMultiTableInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloMultiTableInputFormatIT.java
new file mode 100644
index 0000000..1694f6b
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloMultiTableInputFormatIT.java
@@ -0,0 +1,161 @@
+/*
+ * 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.test.mapreduce;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+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.mapreduce.AccumuloMultiTableInputFormat;
+import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
+import org.apache.accumulo.core.client.mapreduce.RangeInputSplit;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+
+public class AccumuloMultiTableInputFormatIT extends AccumuloClusterHarness {
+
+  private static AssertionError e1 = null;
+  private static AssertionError e2 = null;
+
+  private static class MRTester extends Configured implements Tool {
+
+    private static class TestMapper extends Mapper<Key,Value,Key,Value> {
+      Key key = null;
+      int count = 0;
+
+      @Override
+      protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
+        try {
+          String tableName = ((RangeInputSplit) context.getInputSplit()).getTableName();
+          if (key != null)
+            assertEquals(key.getRow().toString(), new String(v.get()));
+          assertEquals(new Text(String.format("%s_%09x", tableName, count + 1)), k.getRow());
+          assertEquals(String.format("%s_%09x", tableName, count), new String(v.get()));
+        } catch (AssertionError e) {
+          e1 = e;
+        }
+        key = new Key(k);
+        count++;
+      }
+
+      @Override
+      protected void cleanup(Context context) throws IOException, InterruptedException {
+        try {
+          assertEquals(100, count);
+        } catch (AssertionError e) {
+          e2 = e;
+        }
+      }
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+      if (args.length != 2) {
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <table1> <table2>");
+      }
+
+      String user = getAdminPrincipal();
+      AuthenticationToken pass = getAdminToken();
+      String table1 = args[0];
+      String table2 = args[1];
+
+      Job job = Job.getInstance(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
+      job.setJarByClass(this.getClass());
+
+      job.setInputFormatClass(AccumuloMultiTableInputFormat.class);
+
+      AccumuloMultiTableInputFormat.setConnectorInfo(job, user, pass);
+
+      InputTableConfig tableConfig1 = new InputTableConfig();
+      InputTableConfig tableConfig2 = new InputTableConfig();
+
+      Map<String,InputTableConfig> configMap = new HashMap<String,InputTableConfig>();
+      configMap.put(table1, tableConfig1);
+      configMap.put(table2, tableConfig2);
+
+      AccumuloMultiTableInputFormat.setInputTableConfigs(job, configMap);
+      AccumuloMultiTableInputFormat.setZooKeeperInstance(job, getCluster().getClientConfig());
+
+      job.setMapperClass(TestMapper.class);
+      job.setMapOutputKeyClass(Key.class);
+      job.setMapOutputValueClass(Value.class);
+      job.setOutputFormatClass(NullOutputFormat.class);
+
+      job.setNumReduceTasks(0);
+
+      job.waitForCompletion(true);
+
+      return job.isSuccessful() ? 0 : 1;
+    }
+
+    public static void main(String[] args) throws Exception {
+      Configuration conf = new Configuration();
+      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
+      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
+    }
+  }
+
+  /**
+   * Generate incrementing counts and attach table name to the key/value so that order and multi-table data can be verified.
+   */
+  @Test
+  public void testMap() throws Exception {
+    String[] tableNames = getUniqueNames(2);
+    String table1 = tableNames[0];
+    String table2 = tableNames[1];
+    Connector c = getConnector();
+    c.tableOperations().create(table1);
+    c.tableOperations().create(table2);
+    BatchWriter bw = c.createBatchWriter(table1, new BatchWriterConfig());
+    BatchWriter bw2 = c.createBatchWriter(table2, new BatchWriterConfig());
+    for (int i = 0; i < 100; i++) {
+      Mutation t1m = new Mutation(new Text(String.format("%s_%09x", table1, i + 1)));
+      t1m.put(new Text(), new Text(), new Value(String.format("%s_%09x", table1, i).getBytes()));
+      bw.addMutation(t1m);
+      Mutation t2m = new Mutation(new Text(String.format("%s_%09x", table2, i + 1)));
+      t2m.put(new Text(), new Text(), new Value(String.format("%s_%09x", table2, i).getBytes()));
+      bw2.addMutation(t2m);
+    }
+    bw.close();
+    bw2.close();
+
+    MRTester.main(new String[] {table1, table2});
+    assertNull(e1);
+    assertNull(e2);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloOutputFormatIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloOutputFormatIT.java
new file mode 100644
index 0000000..c9307ae
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloOutputFormatIT.java
@@ -0,0 +1,153 @@
+/*
+ * 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.test.mapreduce;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map.Entry;
+
+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.Scanner;
+import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
+import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+
+public class AccumuloOutputFormatIT extends AccumuloClusterHarness {
+  private static AssertionError e1 = null;
+
+  private static class MRTester extends Configured implements Tool {
+    private static class TestMapper extends Mapper<Key,Value,Text,Mutation> {
+      Key key = null;
+      int count = 0;
+
+      @Override
+      protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
+        try {
+          if (key != null)
+            assertEquals(key.getRow().toString(), new String(v.get()));
+          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
+          assertEquals(new String(v.get()), String.format("%09x", count));
+        } catch (AssertionError e) {
+          e1 = e;
+        }
+        key = new Key(k);
+        count++;
+      }
+
+      @Override
+      protected void cleanup(Context context) throws IOException, InterruptedException {
+        Mutation m = new Mutation("total");
+        m.put("", "", Integer.toString(count));
+        context.write(new Text(), m);
+      }
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+      if (args.length != 2) {
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <inputtable> <outputtable>");
+      }
+
+      String user = getAdminPrincipal();
+      AuthenticationToken pass = getAdminToken();
+      String table1 = args[0];
+      String table2 = args[1];
+
+      Job job = Job.getInstance(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
+      job.setJarByClass(this.getClass());
+
+      job.setInputFormatClass(AccumuloInputFormat.class);
+
+      AccumuloInputFormat.setConnectorInfo(job, user, pass);
+      AccumuloInputFormat.setInputTableName(job, table1);
+      AccumuloInputFormat.setZooKeeperInstance(job, getCluster().getClientConfig());
+
+      job.setMapperClass(TestMapper.class);
+      job.setMapOutputKeyClass(Key.class);
+      job.setMapOutputValueClass(Value.class);
+      job.setOutputFormatClass(AccumuloOutputFormat.class);
+      job.setOutputKeyClass(Text.class);
+      job.setOutputValueClass(Mutation.class);
+
+      AccumuloOutputFormat.setConnectorInfo(job, user, pass);
+      AccumuloOutputFormat.setCreateTables(job, false);
+      AccumuloOutputFormat.setDefaultTableName(job, table2);
+      AccumuloOutputFormat.setZooKeeperInstance(job, getCluster().getClientConfig());
+
+      job.setNumReduceTasks(0);
+
+      job.waitForCompletion(true);
+
+      return job.isSuccessful() ? 0 : 1;
+    }
+
+    public static void main(String[] args) throws Exception {
+      Configuration conf = new Configuration();
+      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
+      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
+    }
+  }
+
+  @Test
+  public void testMR() throws Exception {
+    String[] tableNames = getUniqueNames(2);
+    String table1 = tableNames[0];
+    String table2 = tableNames[1];
+    Connector c = getConnector();
+    c.tableOperations().create(table1);
+    c.tableOperations().create(table2);
+    BatchWriter bw = c.createBatchWriter(table1, new BatchWriterConfig());
+    for (int i = 0; i < 100; i++) {
+      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
+      bw.addMutation(m);
+    }
+    bw.close();
+
+    MRTester.main(new String[] {table1, table2});
+    assertNull(e1);
+
+    Scanner scanner = c.createScanner(table2, new Authorizations());
+    Iterator<Entry<Key,Value>> iter = scanner.iterator();
+    assertTrue(iter.hasNext());
+    Entry<Key,Value> entry = iter.next();
+    assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
+    assertFalse(iter.hasNext());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloRowInputFormatIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloRowInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloRowInputFormatIT.java
new file mode 100644
index 0000000..617a8c2
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloRowInputFormatIT.java
@@ -0,0 +1,197 @@
+/*
+ * 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.test.mapreduce;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map.Entry;
+
+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.MutationsRejectedException;
+import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
+import org.apache.accumulo.core.client.mapreduce.AccumuloRowInputFormat;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.KeyValue;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.util.PeekingIterator;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class AccumuloRowInputFormatIT extends AccumuloClusterHarness {
+
+  private static final String ROW1 = "row1";
+  private static final String ROW2 = "row2";
+  private static final String ROW3 = "row3";
+  private static final String COLF1 = "colf1";
+  private static List<Entry<Key,Value>> row1;
+  private static List<Entry<Key,Value>> row2;
+  private static List<Entry<Key,Value>> row3;
+  private static AssertionError e1 = null;
+  private static AssertionError e2 = null;
+
+  @BeforeClass
+  public static void prepareRows() {
+    row1 = new ArrayList<Entry<Key,Value>>();
+    row1.add(new KeyValue(new Key(ROW1, COLF1, "colq1"), "v1".getBytes()));
+    row1.add(new KeyValue(new Key(ROW1, COLF1, "colq2"), "v2".getBytes()));
+    row1.add(new KeyValue(new Key(ROW1, "colf2", "colq3"), "v3".getBytes()));
+    row2 = new ArrayList<Entry<Key,Value>>();
+    row2.add(new KeyValue(new Key(ROW2, COLF1, "colq4"), "v4".getBytes()));
+    row3 = new ArrayList<Entry<Key,Value>>();
+    row3.add(new KeyValue(new Key(ROW3, COLF1, "colq5"), "v5".getBytes()));
+  }
+
+  private static void checkLists(final List<Entry<Key,Value>> first, final Iterator<Entry<Key,Value>> second) {
+    int entryIndex = 0;
+    while (second.hasNext()) {
+      final Entry<Key,Value> entry = second.next();
+      assertEquals("Keys should be equal", first.get(entryIndex).getKey(), entry.getKey());
+      assertEquals("Values should be equal", first.get(entryIndex).getValue(), entry.getValue());
+      entryIndex++;
+    }
+  }
+
+  private static void insertList(final BatchWriter writer, final List<Entry<Key,Value>> list) throws MutationsRejectedException {
+    for (Entry<Key,Value> e : list) {
+      final Key key = e.getKey();
+      final Mutation mutation = new Mutation(key.getRow());
+      ColumnVisibility colVisibility = new ColumnVisibility(key.getColumnVisibility());
+      mutation.put(key.getColumnFamily(), key.getColumnQualifier(), colVisibility, key.getTimestamp(), e.getValue());
+      writer.addMutation(mutation);
+    }
+  }
+
+  private static class MRTester extends Configured implements Tool {
+    private static class TestMapper extends Mapper<Text,PeekingIterator<Entry<Key,Value>>,Key,Value> {
+      int count = 0;
+
+      @Override
+      protected void map(Text k, PeekingIterator<Entry<Key,Value>> v, Context context) throws IOException, InterruptedException {
+        try {
+          switch (count) {
+            case 0:
+              assertEquals("Current key should be " + ROW1, new Text(ROW1), k);
+              checkLists(row1, v);
+              break;
+            case 1:
+              assertEquals("Current key should be " + ROW2, new Text(ROW2), k);
+              checkLists(row2, v);
+              break;
+            case 2:
+              assertEquals("Current key should be " + ROW3, new Text(ROW3), k);
+              checkLists(row3, v);
+              break;
+            default:
+              assertTrue(false);
+          }
+        } catch (AssertionError e) {
+          e1 = e;
+        }
+        count++;
+      }
+
+      @Override
+      protected void cleanup(Context context) throws IOException, InterruptedException {
+        try {
+          assertEquals(3, count);
+        } catch (AssertionError e) {
+          e2 = e;
+        }
+      }
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+      if (args.length != 1) {
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <table>");
+      }
+
+      String user = getAdminPrincipal();
+      AuthenticationToken pass = getAdminToken();
+      String table = args[0];
+
+      Job job = Job.getInstance(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
+      job.setJarByClass(this.getClass());
+
+      job.setInputFormatClass(AccumuloRowInputFormat.class);
+
+      AccumuloInputFormat.setConnectorInfo(job, user, pass);
+      AccumuloInputFormat.setInputTableName(job, table);
+      AccumuloRowInputFormat.setZooKeeperInstance(job, getCluster().getClientConfig());
+
+      job.setMapperClass(TestMapper.class);
+      job.setMapOutputKeyClass(Key.class);
+      job.setMapOutputValueClass(Value.class);
+      job.setOutputFormatClass(NullOutputFormat.class);
+
+      job.setNumReduceTasks(0);
+
+      job.waitForCompletion(true);
+
+      return job.isSuccessful() ? 0 : 1;
+    }
+
+    public static void main(String[] args) throws Exception {
+      Configuration conf = new Configuration();
+      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
+      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
+    }
+  }
+
+  @Test
+  public void test() throws Exception {
+    final Connector conn = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    conn.tableOperations().create(tableName);
+    BatchWriter writer = null;
+    try {
+      writer = conn.createBatchWriter(tableName, new BatchWriterConfig());
+      insertList(writer, row1);
+      insertList(writer, row2);
+      insertList(writer, row3);
+    } finally {
+      if (writer != null) {
+        writer.close();
+      }
+    }
+    MRTester.main(new String[] {tableName});
+    assertNull(e1);
+    assertNull(e2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java b/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java
new file mode 100644
index 0000000..f5deebb
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/mapreduce/TokenFileIT.java
@@ -0,0 +1,167 @@
+/*
+ * 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.test.mapreduce;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.Iterator;
+import java.util.Map.Entry;
+
+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.Scanner;
+import org.apache.accumulo.core.client.impl.Credentials;
+import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
+import org.apache.accumulo.core.client.mapreduce.AccumuloOutputFormat;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TokenFileIT extends AccumuloClusterHarness {
+  private static AssertionError e1 = null;
+
+  private static class MRTokenFileTester extends Configured implements Tool {
+    private static class TestMapper extends Mapper<Key,Value,Text,Mutation> {
+      Key key = null;
+      int count = 0;
+
+      @Override
+      protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
+        try {
+          if (key != null)
+            assertEquals(key.getRow().toString(), new String(v.get()));
+          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
+          assertEquals(new String(v.get()), String.format("%09x", count));
+        } catch (AssertionError e) {
+          e1 = e;
+        }
+        key = new Key(k);
+        count++;
+      }
+
+      @Override
+      protected void cleanup(Context context) throws IOException, InterruptedException {
+        Mutation m = new Mutation("total");
+        m.put("", "", Integer.toString(count));
+        context.write(new Text(), m);
+      }
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+      if (args.length != 3) {
+        throw new IllegalArgumentException("Usage : " + MRTokenFileTester.class.getName() + " <token file> <inputtable> <outputtable>");
+      }
+
+      String user = getAdminPrincipal();
+      String tokenFile = args[0];
+      String table1 = args[1];
+      String table2 = args[2];
+
+      Job job = Job.getInstance(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
+      job.setJarByClass(this.getClass());
+
+      job.setInputFormatClass(AccumuloInputFormat.class);
+
+      AccumuloInputFormat.setConnectorInfo(job, user, tokenFile);
+      AccumuloInputFormat.setInputTableName(job, table1);
+      AccumuloInputFormat.setZooKeeperInstance(job, getCluster().getClientConfig());
+
+      job.setMapperClass(TestMapper.class);
+      job.setMapOutputKeyClass(Key.class);
+      job.setMapOutputValueClass(Value.class);
+      job.setOutputFormatClass(AccumuloOutputFormat.class);
+      job.setOutputKeyClass(Text.class);
+      job.setOutputValueClass(Mutation.class);
+
+      AccumuloOutputFormat.setConnectorInfo(job, user, tokenFile);
+      AccumuloOutputFormat.setCreateTables(job, false);
+      AccumuloOutputFormat.setDefaultTableName(job, table2);
+      AccumuloOutputFormat.setZooKeeperInstance(job, getCluster().getClientConfig());
+
+      job.setNumReduceTasks(0);
+
+      job.waitForCompletion(true);
+
+      return job.isSuccessful() ? 0 : 1;
+    }
+
+    public static void main(String[] args) throws Exception {
+      Configuration conf = CachedConfiguration.getInstance();
+      conf.set("hadoop.tmp.dir", new File(args[0]).getParent());
+      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
+      assertEquals(0, ToolRunner.run(conf, new MRTokenFileTester(), args));
+    }
+  }
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
+
+  @Test
+  public void testMR() throws Exception {
+    String[] tableNames = getUniqueNames(2);
+    String table1 = tableNames[0];
+    String table2 = tableNames[1];
+    Connector c = getConnector();
+    c.tableOperations().create(table1);
+    c.tableOperations().create(table2);
+    BatchWriter bw = c.createBatchWriter(table1, new BatchWriterConfig());
+    for (int i = 0; i < 100; i++) {
+      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
+      bw.addMutation(m);
+    }
+    bw.close();
+
+    File tf = folder.newFile("root_test.pw");
+    PrintStream out = new PrintStream(tf);
+    String outString = new Credentials(getAdminPrincipal(), getAdminToken()).serialize();
+    out.println(outString);
+    out.close();
+
+    MRTokenFileTester.main(new String[] {tf.getAbsolutePath(), table1, table2});
+    assertNull(e1);
+
+    Scanner scanner = c.createScanner(table2, new Authorizations());
+    Iterator<Entry<Key,Value>> iter = scanner.iterator();
+    assertTrue(iter.hasNext());
+    Entry<Key,Value> entry = iter.next();
+    assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
+    assertFalse(iter.hasNext());
+  }
+}


[11/14] accumulo git commit: ACCUMULO-3920 Deprecate mock components

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
index 4e565ee..ad9f419 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
@@ -33,7 +33,6 @@ 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.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -59,12 +58,13 @@ import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TestName;
 
 public class AccumuloInputFormatTest {
 
   private static final String PREFIX = AccumuloInputFormatTest.class.getSimpleName();
-  private static final String INSTANCE_NAME = PREFIX + "_mapred_instance";
   private static final String TEST_TABLE_1 = PREFIX + "_mapred_table_1";
 
   private JobConf job;
@@ -74,6 +74,16 @@ public class AccumuloInputFormatTest {
     System.setProperty("hadoop.tmp.dir", System.getProperty("user.dir") + "/target/hadoop-tmp");
   }
 
+  @Rule
+  public TestName test = new TestName();
+
+  private Instance inst;
+
+  @Before
+  public void setupInstance() throws Exception {
+    inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
+  }
+
   @Before
   public void createJob() {
     job = new JobConf();
@@ -242,13 +252,14 @@ public class AccumuloInputFormatTest {
     @Override
     public int run(String[] args) throws Exception {
 
-      if (args.length != 3) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table>");
+      if (args.length != 4) {
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table> <instanceName>");
       }
 
       String user = args[0];
       String pass = args[1];
       String table = args[2];
+      String instanceName = args[3];
 
       JobConf job = new JobConf(getConf());
       job.setJarByClass(this.getClass());
@@ -257,7 +268,7 @@ public class AccumuloInputFormatTest {
 
       AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
       AccumuloInputFormat.setInputTableName(job, table);
-      AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
+      AccumuloInputFormat.setMockInstance(job, instanceName);
 
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
@@ -278,8 +289,7 @@ public class AccumuloInputFormatTest {
 
   @Test
   public void testMap() throws Exception {
-    MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
-    Connector c = mockInstance.getConnector("root", new PasswordToken(""));
+    Connector c = inst.getConnector("root", new PasswordToken(""));
     c.tableOperations().create(TEST_TABLE_1);
     BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
     for (int i = 0; i < 100; i++) {
@@ -289,7 +299,7 @@ public class AccumuloInputFormatTest {
     }
     bw.close();
 
-    MRTester.main("root", "", TEST_TABLE_1);
+    MRTester.main("root", "", TEST_TABLE_1, inst.getInstanceName());
     assertNull(e1);
     assertNull(e2);
   }
@@ -298,21 +308,20 @@ public class AccumuloInputFormatTest {
   public void testCorrectRangeInputSplits() throws Exception {
     JobConf job = new JobConf();
 
-    String username = "user", table = "table", instance = "mapred_testCorrectRangeInputSplits";
+    String username = "user", table = "table";
     PasswordToken password = new PasswordToken("password");
     Authorizations auths = new Authorizations("foo");
     Collection<Pair<Text,Text>> fetchColumns = Collections.singleton(new Pair<Text,Text>(new Text("foo"), new Text("bar")));
     boolean isolated = true, localIters = true;
     Level level = Level.WARN;
 
-    Instance inst = new MockInstance(instance);
     Connector connector = inst.getConnector(username, password);
     connector.tableOperations().create(table);
 
     AccumuloInputFormat.setConnectorInfo(job, username, password);
     AccumuloInputFormat.setInputTableName(job, table);
     AccumuloInputFormat.setScanAuthorizations(job, auths);
-    AccumuloInputFormat.setMockInstance(job, instance);
+    AccumuloInputFormat.setMockInstance(job, inst.getInstanceName());
     AccumuloInputFormat.setScanIsolation(job, isolated);
     AccumuloInputFormat.setLocalIterators(job, localIters);
     AccumuloInputFormat.fetchColumns(job, fetchColumns);
@@ -334,7 +343,7 @@ public class AccumuloInputFormatTest {
     Assert.assertEquals(table, risplit.getTableName());
     Assert.assertEquals(password, risplit.getToken());
     Assert.assertEquals(auths, risplit.getAuths());
-    Assert.assertEquals(instance, risplit.getInstanceName());
+    Assert.assertEquals(inst.getInstanceName(), risplit.getInstanceName());
     Assert.assertEquals(isolated, risplit.isIsolatedScan());
     Assert.assertEquals(localIters, risplit.usesLocalIterators());
     Assert.assertEquals(fetchColumns, risplit.getFetchedColumns());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
index a5545ee..f9cca65 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
@@ -28,9 +28,9 @@ import java.util.Map;
 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.mapreduce.InputTableConfig;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -142,7 +142,7 @@ public class AccumuloMultiTableInputFormatTest {
 
   @Test
   public void testMap() throws Exception {
-    MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
+    Instance mockInstance = new org.apache.accumulo.core.client.mock.MockInstance(INSTANCE_NAME);
     Connector c = mockInstance.getConnector("root", new PasswordToken(""));
     c.tableOperations().create(TEST_TABLE_1);
     c.tableOperations().create(TEST_TABLE_2);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java
index fa12227..d1f4c4c 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java
@@ -31,8 +31,8 @@ import java.util.concurrent.TimeUnit;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -181,7 +181,7 @@ public class AccumuloOutputFormatTest {
 
   @Test
   public void testMR() throws Exception {
-    MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
+    Instance mockInstance = new org.apache.accumulo.core.client.mock.MockInstance(INSTANCE_NAME);
     Connector c = mockInstance.getConnector("root", new PasswordToken(""));
     c.tableOperations().create(TEST_TABLE_1);
     c.tableOperations().create(TEST_TABLE_2);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java
index 4a52c19..3d59149 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java
@@ -30,8 +30,8 @@ import java.util.Map.Entry;
 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.MutationsRejectedException;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyValue;
@@ -50,11 +50,12 @@ import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.lib.NullOutputFormat;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TestName;
 
 public class AccumuloRowInputFormatTest {
   private static final String PREFIX = AccumuloRowInputFormatTest.class.getSimpleName();
-  private static final String INSTANCE_NAME = PREFIX + "_mapred_instance";
   private static final String TEST_TABLE_1 = PREFIX + "_mapred_table_1";
 
   private static final String ROW1 = "row1";
@@ -152,13 +153,14 @@ public class AccumuloRowInputFormatTest {
     @Override
     public int run(String[] args) throws Exception {
 
-      if (args.length != 3) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table>");
+      if (args.length != 4) {
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table> <instanceName>");
       }
 
       String user = args[0];
       String pass = args[1];
       String table = args[2];
+      String instanceName = args[3];
 
       JobConf job = new JobConf(getConf());
       job.setJarByClass(this.getClass());
@@ -167,7 +169,7 @@ public class AccumuloRowInputFormatTest {
 
       AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
       AccumuloInputFormat.setInputTableName(job, table);
-      AccumuloRowInputFormat.setMockInstance(job, INSTANCE_NAME);
+      AccumuloRowInputFormat.setMockInstance(job, instanceName);
 
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
@@ -186,10 +188,13 @@ public class AccumuloRowInputFormatTest {
     }
   }
 
+  @Rule
+  public TestName test = new TestName();
+
   @Test
   public void test() throws Exception {
-    final MockInstance instance = new MockInstance(INSTANCE_NAME);
-    final Connector conn = instance.getConnector("root", new PasswordToken(""));
+    final Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
+    final Connector conn = inst.getConnector("root", new PasswordToken(""));
     conn.tableOperations().create(TEST_TABLE_1);
     BatchWriter writer = null;
     try {
@@ -202,7 +207,7 @@ public class AccumuloRowInputFormatTest {
         writer.close();
       }
     }
-    MRTester.main(new String[] {"root", "", TEST_TABLE_1});
+    MRTester.main(new String[] {"root", "", TEST_TABLE_1, inst.getInstanceName()});
     assertNull(e1);
     assertNull(e2);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java
index f567454..6c75ec2 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/RangeInputSplitTest.java
@@ -33,6 +33,7 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.iterators.user.SummingCombiner;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
@@ -86,7 +87,7 @@ public class RangeInputSplitTest {
     split.setToken(new PasswordToken("password"));
     split.setPrincipal("root");
     split.setInstanceName("instance");
-    split.setMockInstance(true);
+    DeprecationUtil.setMockInstance(split, true);
     split.setZooKeepers("localhost");
     split.setIterators(iterators);
     split.setLogLevel(Level.WARN);
@@ -112,7 +113,7 @@ public class RangeInputSplitTest {
     Assert.assertEquals(split.getToken(), newSplit.getToken());
     Assert.assertEquals(split.getPrincipal(), newSplit.getPrincipal());
     Assert.assertEquals(split.getInstanceName(), newSplit.getInstanceName());
-    Assert.assertEquals(split.isMockInstance(), newSplit.isMockInstance());
+    Assert.assertEquals(DeprecationUtil.isMockInstanceSet(split), DeprecationUtil.isMockInstanceSet(newSplit));
     Assert.assertEquals(split.getZooKeepers(), newSplit.getZooKeepers());
     Assert.assertEquals(split.getIterators(), newSplit.getIterators());
     Assert.assertEquals(split.getLogLevel(), newSplit.getLogLevel());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java
index f025783..7b4ef75 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java
@@ -30,9 +30,9 @@ import java.util.Map.Entry;
 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.Scanner;
 import org.apache.accumulo.core.client.impl.Credentials;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -52,14 +52,11 @@ import org.apache.hadoop.util.ToolRunner;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
 
-/**
- *
- */
 public class TokenFileTest {
   private static AssertionError e1 = null;
   private static final String PREFIX = TokenFileTest.class.getSimpleName();
-  private static final String INSTANCE_NAME = PREFIX + "_mapred_instance";
   private static final String TEST_TABLE_1 = PREFIX + "_mapred_table_1";
   private static final String TEST_TABLE_2 = PREFIX + "_mapred_table_2";
 
@@ -99,14 +96,15 @@ public class TokenFileTest {
     @Override
     public int run(String[] args) throws Exception {
 
-      if (args.length != 4) {
-        throw new IllegalArgumentException("Usage : " + MRTokenFileTester.class.getName() + " <user> <token file> <inputtable> <outputtable>");
+      if (args.length != 5) {
+        throw new IllegalArgumentException("Usage : " + MRTokenFileTester.class.getName() + " <user> <token file> <inputtable> <outputtable> <instanceName>");
       }
 
       String user = args[0];
       String tokenFile = args[1];
       String table1 = args[2];
       String table2 = args[3];
+      String instanceName = args[4];
 
       JobConf job = new JobConf(getConf());
       job.setJarByClass(this.getClass());
@@ -115,7 +113,7 @@ public class TokenFileTest {
 
       AccumuloInputFormat.setConnectorInfo(job, user, tokenFile);
       AccumuloInputFormat.setInputTableName(job, table1);
-      AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
+      AccumuloInputFormat.setMockInstance(job, instanceName);
 
       job.setMapperClass(TestMapper.class);
       job.setMapOutputKeyClass(Key.class);
@@ -127,7 +125,7 @@ public class TokenFileTest {
       AccumuloOutputFormat.setConnectorInfo(job, user, tokenFile);
       AccumuloOutputFormat.setCreateTables(job, false);
       AccumuloOutputFormat.setDefaultTableName(job, table2);
-      AccumuloOutputFormat.setMockInstance(job, INSTANCE_NAME);
+      AccumuloOutputFormat.setMockInstance(job, instanceName);
 
       job.setNumReduceTasks(0);
 
@@ -145,10 +143,13 @@ public class TokenFileTest {
   @Rule
   public TemporaryFolder folder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
 
+  @Rule
+  public TestName test = new TestName();
+
   @Test
   public void testMR() throws Exception {
-    MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
-    Connector c = mockInstance.getConnector("root", new PasswordToken(""));
+    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
+    Connector c = inst.getConnector("root", new PasswordToken(""));
     c.tableOperations().create(TEST_TABLE_1);
     c.tableOperations().create(TEST_TABLE_2);
     BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
@@ -165,7 +166,7 @@ public class TokenFileTest {
     out.println(outString);
     out.close();
 
-    MRTokenFileTester.main(new String[] {"root", tf.getAbsolutePath(), TEST_TABLE_1, TEST_TABLE_2});
+    MRTokenFileTester.main(new String[] {"root", tf.getAbsolutePath(), TEST_TABLE_1, TEST_TABLE_2, inst.getInstanceName()});
     assertNull(e1);
 
     Scanner scanner = c.createScanner(TEST_TABLE_2, new Authorizations());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
index b8b3c47..43b21ce 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
@@ -28,8 +28,8 @@ import java.io.IOException;
 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.mapreduce.lib.impl.FileOutputConfigurator;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -66,7 +66,7 @@ public class AccumuloFileOutputFormatTest {
 
   @BeforeClass
   public static void setup() throws Exception {
-    MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
+    Instance mockInstance = new org.apache.accumulo.core.client.mock.MockInstance(INSTANCE_NAME);
     Connector c = mockInstance.getConnector("root", new PasswordToken(""));
     c.tableOperations().create(EMPTY_TABLE);
     c.tableOperations().create(TEST_TABLE);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
index 83662e8..ff5e5f6 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
@@ -35,7 +35,6 @@ 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.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -57,11 +56,22 @@ import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.log4j.Level;
 import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TestName;
 
 public class AccumuloInputFormatTest {
 
-  private static final String PREFIX = AccumuloInputFormatTest.class.getSimpleName();
+  @Rule
+  public TestName test = new TestName();
+
+  private Instance inst;
+
+  @Before
+  public void setupInstance() throws Exception {
+    inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
+  }
 
   /**
    * Check that the iterator configuration is getting stored in the Job conf correctly.
@@ -280,11 +290,9 @@ public class AccumuloInputFormatTest {
 
   @Test
   public void testMap() throws Exception {
-    final String INSTANCE_NAME = PREFIX + "_mapreduce_instance";
-    final String TEST_TABLE_1 = PREFIX + "_mapreduce_table_1";
+    final String TEST_TABLE_1 = test.getMethodName() + "_mapreduce_table_1";
 
-    MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
-    Connector c = mockInstance.getConnector("root", new PasswordToken(""));
+    Connector c = inst.getConnector("root", new PasswordToken(""));
     c.tableOperations().create(TEST_TABLE_1);
     BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
     for (int i = 0; i < 100; i++) {
@@ -294,18 +302,16 @@ public class AccumuloInputFormatTest {
     }
     bw.close();
 
-    Assert.assertEquals(0, MRTester.main(new String[] {"root", "", TEST_TABLE_1, INSTANCE_NAME, AccumuloInputFormat.class.getName()}));
+    Assert.assertEquals(0, MRTester.main(new String[] {"root", "", TEST_TABLE_1, inst.getInstanceName(), AccumuloInputFormat.class.getCanonicalName()}));
     assertNull(e1);
     assertNull(e2);
   }
 
   @Test
   public void testMapWithBatchScanner() throws Exception {
-    final String INSTANCE_NAME = PREFIX + "_mapreduce_instance";
-    final String TEST_TABLE_2 = PREFIX + "_mapreduce_table_2";
+    final String TEST_TABLE_2 = test.getMethodName() + "_mapreduce_table_2";
 
-    MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
-    Connector c = mockInstance.getConnector("root", new PasswordToken(""));
+    Connector c = inst.getConnector("root", new PasswordToken(""));
     c.tableOperations().create(TEST_TABLE_2);
     BatchWriter bw = c.createBatchWriter(TEST_TABLE_2, new BatchWriterConfig());
     for (int i = 0; i < 100; i++) {
@@ -315,7 +321,8 @@ public class AccumuloInputFormatTest {
     }
     bw.close();
 
-    Assert.assertEquals(0, MRTester.main(new String[] {"root", "", TEST_TABLE_2, INSTANCE_NAME, AccumuloInputFormat.class.getName(), "True"}));
+    Assert
+        .assertEquals(0, MRTester.main(new String[] {"root", "", TEST_TABLE_2, inst.getInstanceName(), AccumuloInputFormat.class.getCanonicalName(), "True"}));
     assertNull(e1);
     assertNull(e2);
   }
@@ -324,21 +331,20 @@ public class AccumuloInputFormatTest {
   public void testCorrectRangeInputSplits() throws Exception {
     Job job = Job.getInstance(new Configuration(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
 
-    String username = "user", table = "table", instance = "mapreduce_testCorrectRangeInputSplits";
+    String username = "user", table = "table";
     PasswordToken password = new PasswordToken("password");
     Authorizations auths = new Authorizations("foo");
     Collection<Pair<Text,Text>> fetchColumns = Collections.singleton(new Pair<Text,Text>(new Text("foo"), new Text("bar")));
     boolean isolated = true, localIters = true;
     Level level = Level.WARN;
 
-    Instance inst = new MockInstance(instance);
     Connector connector = inst.getConnector(username, password);
     connector.tableOperations().create(table);
 
     AccumuloInputFormat.setConnectorInfo(job, username, password);
     AccumuloInputFormat.setInputTableName(job, table);
     AccumuloInputFormat.setScanAuthorizations(job, auths);
-    AccumuloInputFormat.setMockInstance(job, instance);
+    AccumuloInputFormat.setMockInstance(job, inst.getInstanceName());
     AccumuloInputFormat.setScanIsolation(job, isolated);
     AccumuloInputFormat.setLocalIterators(job, localIters);
     AccumuloInputFormat.fetchColumns(job, fetchColumns);
@@ -360,7 +366,7 @@ public class AccumuloInputFormatTest {
     Assert.assertEquals(table, risplit.getTableName());
     Assert.assertEquals(password, risplit.getToken());
     Assert.assertEquals(auths, risplit.getAuths());
-    Assert.assertEquals(instance, risplit.getInstanceName());
+    Assert.assertEquals(inst.getInstanceName(), risplit.getInstanceName());
     Assert.assertEquals(isolated, risplit.isIsolatedScan());
     Assert.assertEquals(localIters, risplit.usesLocalIterators());
     Assert.assertEquals(fetchColumns, risplit.getFetchedColumns());
@@ -372,8 +378,7 @@ public class AccumuloInputFormatTest {
     String user = "testPartialInputSplitUser";
     PasswordToken password = new PasswordToken("");
 
-    MockInstance mockInstance = new MockInstance("testPartialInputSplitDelegationToConfiguration");
-    Connector c = mockInstance.getConnector(user, password);
+    Connector c = inst.getConnector(user, password);
     c.tableOperations().create("testtable");
     BatchWriter bw = c.createBatchWriter("testtable", new BatchWriterConfig());
     for (int i = 0; i < 100; i++) {
@@ -394,8 +399,7 @@ public class AccumuloInputFormatTest {
     String user = "testPartialFailedInputSplit";
     PasswordToken password = new PasswordToken("");
 
-    MockInstance mockInstance = new MockInstance("testPartialFailedInputSplitDelegationToConfiguration");
-    Connector c = mockInstance.getConnector(user, password);
+    Connector c = inst.getConnector(user, password);
     c.tableOperations().create("testtable");
     BatchWriter bw = c.createBatchWriter("testtable", new BatchWriterConfig());
     for (int i = 0; i < 100; i++) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
index b83bfef..f1c3ca9 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
@@ -28,8 +28,8 @@ import java.util.Map;
 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.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -140,7 +140,7 @@ public class AccumuloMultiTableInputFormatTest {
    */
   @Test
   public void testMap() throws Exception {
-    MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
+    Instance mockInstance = new org.apache.accumulo.core.client.mock.MockInstance(INSTANCE_NAME);
     Connector c = mockInstance.getConnector("root", new PasswordToken(""));
     c.tableOperations().create(TEST_TABLE_1);
     c.tableOperations().create(TEST_TABLE_2);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
index 242bba6..c02ca66 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
@@ -31,8 +31,8 @@ import java.util.concurrent.TimeUnit;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -174,7 +174,7 @@ public class AccumuloOutputFormatTest {
 
   @Test
   public void testMR() throws Exception {
-    MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
+    Instance mockInstance = new org.apache.accumulo.core.client.mock.MockInstance(INSTANCE_NAME);
     Connector c = mockInstance.getConnector("root", new PasswordToken(""));
     c.tableOperations().create(TEST_TABLE_1);
     c.tableOperations().create(TEST_TABLE_2);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
index 2c8bfb1..8df9d0f 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
@@ -30,8 +30,8 @@ import java.util.Map.Entry;
 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.MutationsRejectedException;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyValue;
@@ -183,7 +183,7 @@ public class AccumuloRowInputFormatTest {
 
   @Test
   public void test() throws Exception {
-    final MockInstance instance = new MockInstance(INSTANCE_NAME);
+    final Instance instance = new org.apache.accumulo.core.client.mock.MockInstance(INSTANCE_NAME);
     final Connector conn = instance.getConnector("root", new PasswordToken(""));
     conn.tableOperations().create(TEST_TABLE_1);
     BatchWriter writer = null;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
index 833e594..1cf8c55 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplitTest.java
@@ -33,6 +33,7 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.iterators.user.SummingCombiner;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
@@ -89,7 +90,7 @@ public class RangeInputSplitTest {
     split.setToken(new PasswordToken("password"));
     split.setPrincipal("root");
     split.setInstanceName("instance");
-    split.setMockInstance(true);
+    DeprecationUtil.setMockInstance(split, true);
     split.setZooKeepers("localhost");
     split.setIterators(iterators);
     split.setLogLevel(Level.WARN);
@@ -116,7 +117,7 @@ public class RangeInputSplitTest {
     Assert.assertEquals(split.getToken(), newSplit.getToken());
     Assert.assertEquals(split.getPrincipal(), newSplit.getPrincipal());
     Assert.assertEquals(split.getInstanceName(), newSplit.getInstanceName());
-    Assert.assertEquals(split.isMockInstance(), newSplit.isMockInstance());
+    Assert.assertEquals(DeprecationUtil.isMockInstanceSet(split), DeprecationUtil.isMockInstanceSet(newSplit));
     Assert.assertEquals(split.getZooKeepers(), newSplit.getZooKeepers());
     Assert.assertEquals(split.getIterators(), newSplit.getIterators());
     Assert.assertEquals(split.getLogLevel(), newSplit.getLogLevel());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
index 8f49751..825d905 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
@@ -30,9 +30,9 @@ import java.util.Map.Entry;
 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.Scanner;
 import org.apache.accumulo.core.client.impl.Credentials;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -140,7 +140,7 @@ public class TokenFileTest {
 
   @Test
   public void testMR() throws Exception {
-    MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
+    Instance mockInstance = new org.apache.accumulo.core.client.mock.MockInstance(INSTANCE_NAME);
     Connector c = mockInstance.getConnector("root", new PasswordToken(""));
     c.tableOperations().create(TEST_TABLE_1);
     c.tableOperations().create(TEST_TABLE_2);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/impl/BatchInputSplitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/impl/BatchInputSplitTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/impl/BatchInputSplitTest.java
index 4f3caf0..74c3438 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/impl/BatchInputSplitTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/impl/BatchInputSplitTest.java
@@ -27,13 +27,13 @@ import java.util.HashSet;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.mapreduce.impl.BatchInputSplit;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.iterators.user.SummingCombiner;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
@@ -88,7 +88,7 @@ public class BatchInputSplitTest {
     split.setFetchedColumns(fetchedColumns);
     split.setToken(new PasswordToken("password"));
     split.setPrincipal("root");
-    split.setMockInstance(true);
+    DeprecationUtil.setMockInstance(split, true);
     split.setInstanceName("instance");
     split.setZooKeepers("localhost");
     split.setIterators(iterators);
@@ -113,7 +113,7 @@ public class BatchInputSplitTest {
     Assert.assertEquals(split.getToken(), newSplit.getToken());
     Assert.assertEquals(split.getPrincipal(), newSplit.getPrincipal());
     Assert.assertEquals(split.getInstanceName(), newSplit.getInstanceName());
-    Assert.assertEquals(split.isMockInstance(), newSplit.isMockInstance());
+    Assert.assertEquals(DeprecationUtil.isMockInstanceSet(split), DeprecationUtil.isMockInstanceSet(newSplit));
     Assert.assertEquals(split.getZooKeepers(), newSplit.getZooKeepers());
     Assert.assertEquals(split.getIterators(), newSplit.getIterators());
     Assert.assertEquals(split.getLogLevel(), newSplit.getLogLevel());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java
index 751421a..a7e5e0a 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java
@@ -26,7 +26,6 @@ import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
@@ -100,15 +99,17 @@ public class ConfiguratorBaseTest {
     // assertEquals(1234000, ((ZooKeeperInstance) instance).getZooKeepersSessionTimeOut());
   }
 
+  @SuppressWarnings("deprecation")
   @Test
   public void testSetMockInstance() {
+    Class<?> mockClass = org.apache.accumulo.core.client.mock.MockInstance.class;
     Configuration conf = new Configuration();
     ConfiguratorBase.setMockInstance(this.getClass(), conf, "testInstanceName");
     assertEquals("testInstanceName", conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), ConfiguratorBase.InstanceOpts.NAME)));
     assertEquals(null, conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), ConfiguratorBase.InstanceOpts.ZOO_KEEPERS)));
-    assertEquals(MockInstance.class.getSimpleName(), conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), ConfiguratorBase.InstanceOpts.TYPE)));
+    assertEquals(mockClass.getSimpleName(), conf.get(ConfiguratorBase.enumToConfKey(this.getClass(), ConfiguratorBase.InstanceOpts.TYPE)));
     Instance instance = ConfiguratorBase.getInstance(this.getClass(), conf);
-    assertEquals(MockInstance.class.getName(), instance.getClass().getName());
+    assertEquals(mockClass.getName(), instance.getClass().getName());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java b/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java
index 980498e..4a78a4b 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mock/MockConnectorTest.java
@@ -53,6 +53,7 @@ import org.junit.Test;
 
 import com.google.common.collect.Iterators;
 
+@Deprecated
 public class MockConnectorTest {
   Random random = new Random();
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java b/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java
index 308152e..c1f39e2 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mock/MockNamespacesTest.java
@@ -24,7 +24,6 @@ import static org.junit.Assert.fail;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.Map.Entry;
-import java.util.Random;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -47,11 +46,24 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.Filter;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.security.Authorizations;
+import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TestName;
 
+@Deprecated
 public class MockNamespacesTest {
 
-  Random random = new Random();
+  @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"));
+  }
 
   /**
    * This test creates a table without specifying a namespace. In this case, it puts the table into the default namespace.
@@ -59,12 +71,10 @@ public class MockNamespacesTest {
   @Test
   public void testDefaultNamespace() throws Exception {
     String tableName = "test";
-    Instance instance = new MockInstance("default");
-    Connector c = instance.getConnector("user", new PasswordToken("pass"));
 
-    assertTrue(c.namespaceOperations().exists(Namespaces.DEFAULT_NAMESPACE));
-    c.tableOperations().create(tableName);
-    assertTrue(c.tableOperations().exists(tableName));
+    assertTrue(conn.namespaceOperations().exists(Namespaces.DEFAULT_NAMESPACE));
+    conn.tableOperations().create(tableName);
+    assertTrue(conn.tableOperations().exists(tableName));
   }
 
   /**
@@ -78,38 +88,35 @@ public class MockNamespacesTest {
     String tableName1 = namespace + ".table1";
     String tableName2 = namespace + ".table2";
 
-    Instance instance = new MockInstance("createdelete");
-    Connector c = instance.getConnector("user", new PasswordToken("pass"));
-
-    c.namespaceOperations().create(namespace);
-    assertTrue(c.namespaceOperations().exists(namespace));
+    conn.namespaceOperations().create(namespace);
+    assertTrue(conn.namespaceOperations().exists(namespace));
 
-    c.tableOperations().create(tableName1);
-    assertTrue(c.tableOperations().exists(tableName1));
+    conn.tableOperations().create(tableName1);
+    assertTrue(conn.tableOperations().exists(tableName1));
 
-    c.tableOperations().create(tableName2);
-    assertTrue(c.tableOperations().exists(tableName2));
+    conn.tableOperations().create(tableName2);
+    assertTrue(conn.tableOperations().exists(tableName2));
 
     // deleting
     try {
       // can't delete a namespace with tables in it
-      c.namespaceOperations().delete(namespace);
+      conn.namespaceOperations().delete(namespace);
       fail();
     } catch (NamespaceNotEmptyException e) {
       // ignore, supposed to happen
     }
-    assertTrue(c.namespaceOperations().exists(namespace));
-    assertTrue(c.tableOperations().exists(tableName1));
-    assertTrue(c.tableOperations().exists(tableName2));
-
-    c.tableOperations().delete(tableName2);
-    assertTrue(!c.tableOperations().exists(tableName2));
-    assertTrue(c.namespaceOperations().exists(namespace));
-
-    c.tableOperations().delete(tableName1);
-    assertTrue(!c.tableOperations().exists(tableName1));
-    c.namespaceOperations().delete(namespace);
-    assertTrue(!c.namespaceOperations().exists(namespace));
+    assertTrue(conn.namespaceOperations().exists(namespace));
+    assertTrue(conn.tableOperations().exists(tableName1));
+    assertTrue(conn.tableOperations().exists(tableName2));
+
+    conn.tableOperations().delete(tableName2);
+    assertTrue(!conn.tableOperations().exists(tableName2));
+    assertTrue(conn.namespaceOperations().exists(namespace));
+
+    conn.tableOperations().delete(tableName1);
+    assertTrue(!conn.tableOperations().exists(tableName1));
+    conn.namespaceOperations().delete(namespace);
+    assertTrue(!conn.namespaceOperations().exists(namespace));
   }
 
   /**
@@ -130,51 +137,48 @@ public class MockNamespacesTest {
     String propKey = Property.TABLE_SCAN_MAXMEM.getKey();
     String propVal = "42K";
 
-    Instance instance = new MockInstance("props");
-    Connector c = instance.getConnector("user", new PasswordToken("pass"));
-
-    c.namespaceOperations().create(namespace);
-    c.tableOperations().create(tableName1);
-    c.namespaceOperations().setProperty(namespace, propKey, propVal);
+    conn.namespaceOperations().create(namespace);
+    conn.tableOperations().create(tableName1);
+    conn.namespaceOperations().setProperty(namespace, propKey, propVal);
 
     // check the namespace has the property
-    assertTrue(checkNamespaceHasProp(c, namespace, propKey, propVal));
+    assertTrue(checkNamespaceHasProp(conn, namespace, propKey, propVal));
 
     // check that the table gets it from the namespace
-    assertTrue(checkTableHasProp(c, tableName1, propKey, propVal));
+    assertTrue(checkTableHasProp(conn, tableName1, propKey, propVal));
 
     // test a second table to be sure the first wasn't magical
     // (also, changed the order, the namespace has the property already)
-    c.tableOperations().create(tableName2);
-    assertTrue(checkTableHasProp(c, tableName2, propKey, propVal));
+    conn.tableOperations().create(tableName2);
+    assertTrue(checkTableHasProp(conn, tableName2, propKey, propVal));
 
     // test that table properties override namespace properties
     String propKey2 = Property.TABLE_FILE_MAX.getKey();
     String propVal2 = "42";
     String tablePropVal = "13";
 
-    c.tableOperations().setProperty(tableName2, propKey2, tablePropVal);
-    c.namespaceOperations().setProperty("propchange", propKey2, propVal2);
+    conn.tableOperations().setProperty(tableName2, propKey2, tablePropVal);
+    conn.namespaceOperations().setProperty("propchange", propKey2, propVal2);
 
-    assertTrue(checkTableHasProp(c, tableName2, propKey2, tablePropVal));
+    assertTrue(checkTableHasProp(conn, tableName2, propKey2, tablePropVal));
 
     // now check that you can change the default namespace's properties
     propVal = "13K";
     String tableName = "some_table";
-    c.tableOperations().create(tableName);
-    c.namespaceOperations().setProperty(Namespaces.DEFAULT_NAMESPACE, propKey, propVal);
+    conn.tableOperations().create(tableName);
+    conn.namespaceOperations().setProperty(Namespaces.DEFAULT_NAMESPACE, propKey, propVal);
 
-    assertTrue(checkTableHasProp(c, tableName, propKey, propVal));
+    assertTrue(checkTableHasProp(conn, tableName, propKey, propVal));
 
     // test the properties server-side by configuring an iterator.
     // should not show anything with column-family = 'a'
     String tableName3 = namespace + ".table3";
-    c.tableOperations().create(tableName3);
+    conn.tableOperations().create(tableName3);
 
     IteratorSetting setting = new IteratorSetting(250, "thing", SimpleFilter.class.getName());
-    c.namespaceOperations().attachIterator(namespace, setting);
+    conn.namespaceOperations().attachIterator(namespace, setting);
 
-    BatchWriter bw = c.createBatchWriter(tableName3, new BatchWriterConfig());
+    BatchWriter bw = conn.createBatchWriter(tableName3, new BatchWriterConfig());
     Mutation m = new Mutation("r");
     m.put("a", "b", new Value("abcde".getBytes()));
     bw.addMutation(m);
@@ -197,22 +201,18 @@ public class MockNamespacesTest {
     String tableName1 = "renamed.table1";
     // String tableName2 = "cloned.table2";
 
-    Instance instance = new MockInstance("renameclone");
-    Connector c = instance.getConnector("user", new PasswordToken("pass"));
-
-    c.tableOperations().create(tableName);
-    c.namespaceOperations().create(namespace1);
-    c.namespaceOperations().create(namespace2);
+    conn.tableOperations().create(tableName);
+    conn.namespaceOperations().create(namespace1);
+    conn.namespaceOperations().create(namespace2);
 
-    c.tableOperations().rename(tableName, tableName1);
+    conn.tableOperations().rename(tableName, tableName1);
 
-    assertTrue(c.tableOperations().exists(tableName1));
-    assertTrue(!c.tableOperations().exists(tableName));
+    assertTrue(conn.tableOperations().exists(tableName1));
+    assertTrue(!conn.tableOperations().exists(tableName));
 
     // TODO implement clone in mock
     // c.tableOperations().clone(tableName1, tableName2, false, null, null);
     // assertTrue(c.tableOperations().exists(tableName1)); assertTrue(c.tableOperations().exists(tableName2));
-    return;
   }
 
   /**
@@ -224,18 +224,15 @@ public class MockNamespacesTest {
     String namespace2 = "n2";
     String table = "t";
 
-    Instance instance = new MockInstance("rename");
-    Connector c = instance.getConnector("user", new PasswordToken("pass"));
+    conn.namespaceOperations().create(namespace1);
+    conn.tableOperations().create(namespace1 + "." + table);
 
-    c.namespaceOperations().create(namespace1);
-    c.tableOperations().create(namespace1 + "." + table);
+    conn.namespaceOperations().rename(namespace1, namespace2);
 
-    c.namespaceOperations().rename(namespace1, namespace2);
-
-    assertTrue(!c.namespaceOperations().exists(namespace1));
-    assertTrue(c.namespaceOperations().exists(namespace2));
-    assertTrue(!c.tableOperations().exists(namespace1 + "." + table));
-    assertTrue(c.tableOperations().exists(namespace2 + "." + table));
+    assertTrue(!conn.namespaceOperations().exists(namespace1));
+    assertTrue(conn.namespaceOperations().exists(namespace2));
+    assertTrue(!conn.tableOperations().exists(namespace1 + "." + table));
+    assertTrue(conn.tableOperations().exists(namespace2 + "." + table));
   }
 
   /**
@@ -243,34 +240,31 @@ public class MockNamespacesTest {
    */
   @Test
   public void testNamespaceIterators() throws Exception {
-    Instance instance = new MockInstance("Iterators");
-    Connector c = instance.getConnector("user", new PasswordToken("pass"));
-
     String namespace = "iterator";
     String tableName = namespace + ".table";
     String iter = "thing";
 
-    c.namespaceOperations().create(namespace);
-    c.tableOperations().create(tableName);
+    conn.namespaceOperations().create(namespace);
+    conn.tableOperations().create(tableName);
 
     IteratorSetting setting = new IteratorSetting(250, iter, SimpleFilter.class.getName());
     HashSet<IteratorScope> scope = new HashSet<IteratorScope>();
     scope.add(IteratorScope.scan);
-    c.namespaceOperations().attachIterator(namespace, setting, EnumSet.copyOf(scope));
+    conn.namespaceOperations().attachIterator(namespace, setting, EnumSet.copyOf(scope));
 
-    BatchWriter bw = c.createBatchWriter(tableName, new BatchWriterConfig());
+    BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig());
     Mutation m = new Mutation("r");
     m.put("a", "b", new Value("abcde".getBytes(UTF_8)));
     bw.addMutation(m);
     bw.flush();
 
-    Scanner s = c.createScanner(tableName, Authorizations.EMPTY);
+    Scanner s = conn.createScanner(tableName, Authorizations.EMPTY);
     System.out.println(s.iterator().next());
     // do scanners work correctly in mock?
     // assertTrue(!s.iterator().hasNext());
 
-    assertTrue(c.namespaceOperations().listIterators(namespace).containsKey(iter));
-    c.namespaceOperations().removeIterator(namespace, iter, EnumSet.copyOf(scope));
+    assertTrue(conn.namespaceOperations().listIterators(namespace).containsKey(iter));
+    conn.namespaceOperations().removeIterator(namespace, iter, EnumSet.copyOf(scope));
   }
 
   private boolean checkTableHasProp(Connector c, String t, String propKey, String propVal) throws AccumuloException, TableNotFoundException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/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
index 9733bd3..792e199 100644
--- 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
@@ -59,16 +59,29 @@ 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 {
-    Instance instance = new MockInstance("topstest");
-    Connector conn = instance.getConnector("user", new PasswordToken("pass"));
     String t = "tableName1";
 
     {
@@ -128,8 +141,6 @@ public class MockTableOperationsTest {
 
   @Test
   public void testTableNotFound() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException {
-    Instance instance = new MockInstance("topstest");
-    Connector conn = instance.getConnector("user", new PasswordToken("pass"));
     IteratorSetting setting = new IteratorSetting(100, "myvers", VersioningIterator.class);
     String t = "tableName";
     try {
@@ -161,7 +172,7 @@ public class MockTableOperationsTest {
       Assert.fail();
     } catch (TableNotFoundException e) {}
     try {
-      conn.tableOperations().removeIterator(t, null, null);
+      conn.tableOperations().removeIterator(t, null, EnumSet.noneOf(IteratorScope.class));
       Assert.fail();
     } catch (TableNotFoundException e) {}
     try {
@@ -188,12 +199,10 @@ public class MockTableOperationsTest {
   @Test
   public void testImport() throws Throwable {
     ImportTestFilesAndData dataAndFiles = prepareTestFiles();
-    Instance instance = new MockInstance("foo");
-    Connector connector = instance.getConnector("user", new PasswordToken(new byte[0]));
-    TableOperations tableOperations = connector.tableOperations();
+    TableOperations tableOperations = conn.tableOperations();
     tableOperations.create("a_table");
     tableOperations.importDirectory("a_table", dataAndFiles.importPath.toString(), dataAndFiles.failurePath.toString(), false);
-    Scanner scanner = connector.createScanner("a_table", new Authorizations());
+    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());
@@ -235,18 +244,14 @@ public class MockTableOperationsTest {
 
   @Test(expected = TableNotFoundException.class)
   public void testFailsWithNoTable() throws Throwable {
-    Instance instance = new MockInstance("foo");
-    Connector connector = instance.getConnector("user", new PasswordToken(new byte[0]));
-    TableOperations tableOperations = connector.tableOperations();
+    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 {
-    Instance instance = new MockInstance("foo");
-    Connector connector = instance.getConnector("user", new PasswordToken(new byte[0]));
-    TableOperations tableOperations = connector.tableOperations();
+    TableOperations tableOperations = conn.tableOperations();
     ImportTestFilesAndData testFiles = prepareTestFiles();
     FileSystem fs = testFiles.failurePath.getFileSystem(new Configuration());
     fs.open(testFiles.failurePath.suffix("/something")).close();
@@ -255,11 +260,9 @@ public class MockTableOperationsTest {
 
   @Test
   public void testDeleteRows() throws Exception {
-    Instance instance = new MockInstance("rows");
-    Connector connector = instance.getConnector("user", new PasswordToken("foo".getBytes()));
-    TableOperations to = connector.tableOperations();
+    TableOperations to = conn.tableOperations();
     to.create("test");
-    BatchWriter bw = connector.createBatchWriter("test", new BatchWriterConfig());
+    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++) {
@@ -269,7 +272,7 @@ public class MockTableOperationsTest {
     }
     bw.flush();
     to.deleteRows("test", new Text("1"), new Text("2"));
-    Scanner s = connector.createScanner("test", Authorizations.EMPTY);
+    Scanner s = conn.createScanner("test", Authorizations.EMPTY);
     int oneCnt = 0;
     for (Entry<Key,Value> entry : s) {
       char rowStart = entry.getKey().getRow().toString().charAt(0);
@@ -281,11 +284,9 @@ public class MockTableOperationsTest {
 
   @Test
   public void testDeleteRowsWithNullKeys() throws Exception {
-    Instance instance = new MockInstance("rows");
-    Connector connector = instance.getConnector("user", new PasswordToken("foo"));
-    TableOperations to = connector.tableOperations();
+    TableOperations to = conn.tableOperations();
     to.create("test2");
-    BatchWriter bw = connector.createBatchWriter("test2", new BatchWriterConfig());
+    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++) {
@@ -298,7 +299,7 @@ public class MockTableOperationsTest {
     // test null end
     // will remove rows 4 through 9 (6 * 5 = 30 entries)
     to.deleteRows("test2", new Text("30"), null);
-    Scanner s = connector.createScanner("test2", Authorizations.EMPTY);
+    Scanner s = conn.createScanner("test2", Authorizations.EMPTY);
     int rowCnt = 0;
     for (Entry<Key,Value> entry : s) {
       String rowId = entry.getKey().getRow().toString();
@@ -311,7 +312,7 @@ public class MockTableOperationsTest {
     // test null start
     // will remove 0-1, 10-19, 2
     to.deleteRows("test2", null, new Text("2"));
-    s = connector.createScanner("test2", Authorizations.EMPTY);
+    s = conn.createScanner("test2", Authorizations.EMPTY);
     rowCnt = 0;
     for (Entry<Key,Value> entry : s) {
       char rowStart = entry.getKey().getRow().toString().charAt(0);
@@ -324,7 +325,7 @@ public class MockTableOperationsTest {
     // test null start and end
     // deletes everything still left
     to.deleteRows("test2", null, null);
-    s = connector.createScanner("test2", Authorizations.EMPTY);
+    s = conn.createScanner("test2", Authorizations.EMPTY);
     rowCnt = Iterators.size(s.iterator());
     s.close();
     to.delete("test2");
@@ -334,8 +335,6 @@ public class MockTableOperationsTest {
 
   @Test
   public void testTableIdMap() throws Exception {
-    Instance inst = new MockInstance("testTableIdMap");
-    Connector conn = inst.getConnector("root", new PasswordToken(""));
     TableOperations tops = conn.tableOperations();
     tops.create("foo");
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/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
index b03bda9..4f041c9 100644
--- 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
@@ -31,11 +31,23 @@ 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();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mock/TransformIterator.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mock/TransformIterator.java b/core/src/test/java/org/apache/accumulo/core/client/mock/TransformIterator.java
deleted file mode 100644
index a7e7eef..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mock/TransformIterator.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.client.mock;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.iterators.WrappingIterator;
-import org.apache.hadoop.io.Text;
-
-public 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());
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java
index 365cee4..c3696d4 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java
@@ -16,6 +16,10 @@
  */
 package org.apache.accumulo.core.iterators.user;
 
+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;
@@ -26,14 +30,12 @@ import java.util.Map.Entry;
 import java.util.Random;
 import java.util.TreeMap;
 
-import junit.framework.TestCase;
-
 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.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
@@ -47,13 +49,13 @@ import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.accumulo.core.iterators.system.MultiIterator;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
 
-public class IntersectingIteratorTest extends TestCase {
+public class IntersectingIteratorTest {
 
   private static final Collection<ByteSequence> EMPTY_COL_FAMS = new ArrayList<ByteSequence>();
-  private static final Logger log = Logger.getLogger(IntersectingIterator.class);
   private static IteratorEnvironment env = new DefaultIteratorEnvironment();
 
   TreeMap<Key,Value> map;
@@ -66,10 +68,6 @@ public class IntersectingIteratorTest extends TestCase {
 
   int docid = 0;
 
-  static {
-    log.setLevel(Level.OFF);
-  }
-
   private TreeMap<Key,Value> createSortedMap(float hitRatio, int numRows, int numDocsPerRow, Text[] columnFamilies, Text[] otherColumnFamilies,
       HashSet<Text> docs, Text[] negatedColumns) {
     Random r = new Random();
@@ -130,16 +128,13 @@ public class IntersectingIteratorTest extends TestCase {
     docid = 0;
   }
 
-  public void testNull() {}
-
-  @Override
-  public void setUp() {
-    Logger.getRootLogger().setLevel(Level.ERROR);
-  }
-
   private static final int NUM_ROWS = 10;
   private static final int NUM_DOCIDS = 1000;
 
+  @Rule
+  public TestName test = new TestName();
+
+  @Test
   public void test1() throws IOException {
     columnFamilies = new Text[2];
     columnFamilies[0] = new Text("C");
@@ -168,6 +163,7 @@ public class IntersectingIteratorTest extends TestCase {
     cleanup();
   }
 
+  @Test
   public void test2() throws IOException {
     columnFamilies = new Text[3];
     columnFamilies[0] = new Text("A");
@@ -197,6 +193,7 @@ public class IntersectingIteratorTest extends TestCase {
     cleanup();
   }
 
+  @Test
   public void test3() throws IOException {
     columnFamilies = new Text[6];
     columnFamilies[0] = new Text("C");
@@ -234,6 +231,7 @@ public class IntersectingIteratorTest extends TestCase {
     cleanup();
   }
 
+  @Test
   public void test4() throws IOException {
     columnFamilies = new Text[3];
     notFlags = new boolean[3];
@@ -270,6 +268,7 @@ public class IntersectingIteratorTest extends TestCase {
     cleanup();
   }
 
+  @Test
   public void test6() throws IOException {
     columnFamilies = new Text[1];
     columnFamilies[0] = new Text("C");
@@ -297,9 +296,10 @@ public class IntersectingIteratorTest extends TestCase {
     cleanup();
   }
 
+  @Test
   public void testWithBatchScanner() throws Exception {
     Value empty = new Value(new byte[] {});
-    MockInstance inst = new MockInstance("mockabye");
+    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
     Connector connector = inst.getConnector("user", new PasswordToken("pass"));
     connector.tableOperations().create("index");
     BatchWriter bw = connector.createBatchWriter("index", new BatchWriterConfig());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/iterators/user/RegExFilterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/RegExFilterTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/RegExFilterTest.java
index 2649f90..7a203c8 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/RegExFilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/RegExFilterTest.java
@@ -17,14 +17,14 @@
 package org.apache.accumulo.core.iterators.user;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+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.TreeMap;
 
-import junit.framework.TestCase;
-
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
@@ -35,7 +35,6 @@ import org.apache.accumulo.core.client.IteratorSetting;
 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.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
@@ -46,12 +45,26 @@ import org.apache.accumulo.core.iterators.DefaultIteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TestName;
 
-public class RegExFilterTest extends TestCase {
+public class RegExFilterTest {
 
   private static final Collection<ByteSequence> EMPTY_COL_FAMS = new ArrayList<ByteSequence>();
 
+  private Connector conn;
+
+  @Rule
+  public TestName test = new TestName();
+
+  @Before
+  public void setupInstance() throws Exception {
+    Instance instance = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
+    conn = instance.getConnector("root", new PasswordToken(""));
+  }
+
   private Key nkv(TreeMap<Key,Value> tm, String row, String cf, String cq, String val) {
     Key k = nk(row, cf, cq);
     tm.put(k, new Value(val.getBytes()));
@@ -62,6 +75,7 @@ public class RegExFilterTest extends TestCase {
     return new Key(new Text(row), new Text(cf), new Text(cq));
   }
 
+  @Test
   public void test1() throws IOException {
     TreeMap<Key,Value> tm = new TreeMap<Key,Value>();
 
@@ -263,9 +277,6 @@ public class RegExFilterTest extends TestCase {
     ball[b1.length] = (byte) 0;
     System.arraycopy(b2, 0, ball, b1.length + 1, b2.length);
 
-    Instance instance = new MockInstance();
-    Connector conn = instance.getConnector("root", new PasswordToken(new byte[0]));
-
     conn.tableOperations().create(table);
     BatchWriter bw = conn.createBatchWriter(table, new BatchWriterConfig());
     Mutation m = new Mutation(ball);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
index 7914ec0..6026c14 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
@@ -32,9 +32,9 @@ import java.util.TreeMap;
 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.Scanner;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.ColumnUpdate;
@@ -47,14 +47,24 @@ import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
-
-/**
- *
- */
+import org.junit.rules.TestName;
 
 public class RowFilterTest {
 
+  @Rule
+  public TestName test = new TestName();
+
+  private Connector conn;
+
+  @Before
+  public void setupInstance() throws Exception {
+    Instance instance = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
+    conn = instance.getConnector("", new PasswordToken(""));
+  }
+
   public static class SummingRowFilter extends RowFilter {
 
     @Override
@@ -187,9 +197,6 @@ public class RowFilterTest {
 
   @Test
   public void test1() throws Exception {
-    MockInstance instance = new MockInstance("rft1");
-    Connector conn = instance.getConnector("", new PasswordToken(""));
-
     conn.tableOperations().create("table1");
     BatchWriter bw = conn.createBatchWriter("table1", new BatchWriterConfig());
 
@@ -231,9 +238,6 @@ public class RowFilterTest {
 
   @Test
   public void testChainedRowFilters() throws Exception {
-    MockInstance instance = new MockInstance("rft1");
-    Connector conn = instance.getConnector("", new PasswordToken(""));
-
     conn.tableOperations().create("chained_row_filters");
     BatchWriter bw = conn.createBatchWriter("chained_row_filters", new BatchWriterConfig());
     for (Mutation m : createMutations()) {
@@ -247,9 +251,6 @@ public class RowFilterTest {
 
   @Test
   public void testFilterConjunction() throws Exception {
-    MockInstance instance = new MockInstance("rft1");
-    Connector conn = instance.getConnector("", new PasswordToken(""));
-
     conn.tableOperations().create("filter_conjunction");
     BatchWriter bw = conn.createBatchWriter("filter_conjunction", new BatchWriterConfig());
     for (Mutation m : createMutations()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
index 758f718..e98afd7 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
@@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -37,9 +38,9 @@ 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.Scanner;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.ArrayByteSequence;
@@ -58,7 +59,9 @@ import org.apache.accumulo.core.security.ColumnVisibility;
 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;
 
 public class TransformingIteratorTest {
   private static final String TABLE_NAME = "test_table";
@@ -66,9 +69,12 @@ public class TransformingIteratorTest {
   private Connector connector;
   private Scanner scanner;
 
+  @Rule
+  public TestName test = new TestName();
+
   @Before
   public void setUpMockAccumulo() throws Exception {
-    MockInstance instance = new MockInstance("test");
+    Instance instance = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
     connector = instance.getConnector("user", new PasswordToken("password"));
     connector.securityOperations().changeUserAuthorizations("user", authorizations);
 
@@ -278,9 +284,6 @@ public class TransformingIteratorTest {
 
   @Test
   public void testDeepCopy() throws Exception {
-    MockInstance instance = new MockInstance("test");
-    Connector connector = instance.getConnector("user", new PasswordToken("password"));
-
     connector.tableOperations().create("shard_table");
 
     BatchWriter bw = connector.createBatchWriter("shard_table", new BatchWriterConfig());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/metadata/MetadataServicerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/metadata/MetadataServicerTest.java b/core/src/test/java/org/apache/accumulo/core/metadata/MetadataServicerTest.java
index 0e59025..0a0a940 100644
--- a/core/src/test/java/org/apache/accumulo/core/metadata/MetadataServicerTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/metadata/MetadataServicerTest.java
@@ -21,20 +21,46 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
 
+import java.util.HashMap;
+
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.impl.ClientContext;
-import org.apache.accumulo.core.client.impl.Credentials;
-import org.apache.accumulo.core.client.mock.MockInstance;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.replication.ReplicationTable;
+import org.easymock.EasyMock;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class MetadataServicerTest {
 
+  private static final String userTableName = "tableName";
+  private static final String userTableId = "tableId";
+  private static ClientContext context;
+
+  @BeforeClass
+  public static void setupContext() throws Exception {
+    HashMap<String,String> tableNameToIdMap = new HashMap<>();
+    tableNameToIdMap.put(RootTable.NAME, RootTable.ID);
+    tableNameToIdMap.put(MetadataTable.NAME, MetadataTable.ID);
+    tableNameToIdMap.put(ReplicationTable.NAME, ReplicationTable.ID);
+    tableNameToIdMap.put(userTableName, userTableId);
+
+    context = EasyMock.createMock(ClientContext.class);
+    Connector conn = EasyMock.createMock(Connector.class);
+    Instance inst = EasyMock.createMock(Instance.class);
+    TableOperations tableOps = EasyMock.createMock(TableOperations.class);
+    EasyMock.expect(tableOps.tableIdMap()).andReturn(tableNameToIdMap).anyTimes();
+    EasyMock.expect(conn.tableOperations()).andReturn(tableOps).anyTimes();
+    EasyMock.expect(context.getInstance()).andReturn(inst).anyTimes();
+    EasyMock.expect(context.getConnector()).andReturn(conn).anyTimes();
+    EasyMock.replay(context, conn, inst, tableOps);
+  }
+
   @Test
   public void checkSystemTableIdentifiers() {
     assertNotEquals(RootTable.ID, MetadataTable.ID);
@@ -43,14 +69,6 @@ public class MetadataServicerTest {
 
   @Test
   public void testGetCorrectServicer() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException {
-    String userTableName = "A";
-    MockInstance instance = new MockInstance("metadataTest");
-    Connector connector = instance.getConnector("root", new PasswordToken(""));
-    connector.tableOperations().create(userTableName);
-    String userTableId = connector.tableOperations().tableIdMap().get(userTableName);
-    Credentials credentials = new Credentials("root", new PasswordToken(""));
-    ClientContext context = new ClientContext(instance, credentials, new ClientConfiguration());
-
     MetadataServicer ms = MetadataServicer.forTableId(context, RootTable.ID);
     assertTrue(ms instanceof ServicerForRootTable);
     assertFalse(ms instanceof TableMetadataServicer);
@@ -62,6 +80,12 @@ public class MetadataServicerTest {
     assertEquals(RootTable.NAME, ((TableMetadataServicer) ms).getServicingTableName());
     assertEquals(MetadataTable.ID, ms.getServicedTableId());
 
+    ms = MetadataServicer.forTableId(context, ReplicationTable.ID);
+    assertTrue(ms instanceof ServicerForUserTables);
+    assertTrue(ms instanceof TableMetadataServicer);
+    assertEquals(MetadataTable.NAME, ((TableMetadataServicer) ms).getServicingTableName());
+    assertEquals(ReplicationTable.ID, ms.getServicedTableId());
+
     ms = MetadataServicer.forTableId(context, userTableId);
     assertTrue(ms instanceof ServicerForUserTables);
     assertTrue(ms instanceof TableMetadataServicer);
@@ -79,6 +103,12 @@ public class MetadataServicerTest {
     assertEquals(RootTable.NAME, ((TableMetadataServicer) ms).getServicingTableName());
     assertEquals(MetadataTable.ID, ms.getServicedTableId());
 
+    ms = MetadataServicer.forTableName(context, ReplicationTable.NAME);
+    assertTrue(ms instanceof ServicerForUserTables);
+    assertTrue(ms instanceof TableMetadataServicer);
+    assertEquals(MetadataTable.NAME, ((TableMetadataServicer) ms).getServicingTableName());
+    assertEquals(ReplicationTable.ID, ms.getServicedTableId());
+
     ms = MetadataServicer.forTableName(context, userTableName);
     assertTrue(ms instanceof ServicerForUserTables);
     assertTrue(ms instanceof TableMetadataServicer);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/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 0457caa..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
@@ -30,24 +30,37 @@ 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.mock.MockInstance;
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
 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;
 import org.junit.Test;
+import org.junit.rules.TestName;
 
-/**
- *
- */
 public class CredentialsTest {
 
+  @Rule
+  public TestName test = new TestName();
+
+  private Instance inst;
+
+  @Before
+  public void setupInstance() {
+    inst = EasyMock.createMock(Instance.class);
+    EasyMock.expect(inst.getInstanceID()).andReturn(test.getMethodName()).anyTimes();
+    EasyMock.replay(inst);
+  }
+
   @Test
   public void testToThrift() throws DestroyFailedException {
     // verify thrift serialization
     Credentials creds = new Credentials("test", new PasswordToken("testing"));
-    TCredentials tCreds = creds.toThrift(new MockInstance());
+    TCredentials tCreds = creds.toThrift(inst);
     assertEquals("test", tCreds.getPrincipal());
     assertEquals(PasswordToken.class.getName(), tCreds.getTokenClassName());
     assertArrayEquals(AuthenticationTokenSerializer.serialize(new PasswordToken("testing")), tCreds.getToken());
@@ -55,7 +68,7 @@ public class CredentialsTest {
     // verify that we can't serialize if it's destroyed
     creds.getToken().destroy();
     try {
-      creds.toThrift(new MockInstance());
+      creds.toThrift(inst);
       fail();
     } catch (Exception e) {
       assertTrue(e instanceof RuntimeException);
@@ -67,14 +80,14 @@ public class CredentialsTest {
   @Test
   public void roundtripThrift() throws DestroyFailedException {
     Credentials creds = new Credentials("test", new PasswordToken("testing"));
-    TCredentials tCreds = creds.toThrift(new MockInstance());
+    TCredentials tCreds = creds.toThrift(inst);
     Credentials roundtrip = Credentials.fromThrift(tCreds);
     assertEquals("Roundtrip through thirft changed credentials equality", creds, roundtrip);
   }
 
   @Test
   public void testMockConnector() throws AccumuloException, DestroyFailedException, AccumuloSecurityException {
-    Instance inst = new MockInstance();
+    Instance inst = DeprecationUtil.makeMockInstance(test.getMethodName());
     Connector rootConnector = inst.getConnector("root", new PasswordToken());
     PasswordToken testToken = new PasswordToken("testPass");
     rootConnector.securityOperations().createLocalUser("testUser", testToken);


[10/14] accumulo git commit: ACCUMULO-3920 Deprecate mock components

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/examples/simple/src/test/java/org/apache/accumulo/examples/simple/dirlist/CountTest.java
----------------------------------------------------------------------
diff --git a/examples/simple/src/test/java/org/apache/accumulo/examples/simple/dirlist/CountTest.java b/examples/simple/src/test/java/org/apache/accumulo/examples/simple/dirlist/CountTest.java
index f089d42..492f3e5 100644
--- a/examples/simple/src/test/java/org/apache/accumulo/examples/simple/dirlist/CountTest.java
+++ b/examples/simple/src/test/java/org/apache/accumulo/examples/simple/dirlist/CountTest.java
@@ -16,19 +16,20 @@
  */
 package org.apache.accumulo.examples.simple.dirlist;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
 import java.util.ArrayList;
 import java.util.Map.Entry;
 
-import junit.framework.TestCase;
-
 import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.cli.ClientOpts.Password;
 import org.apache.accumulo.core.cli.ScannerOpts;
 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.Scanner;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
@@ -37,37 +38,43 @@ import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.examples.simple.dirlist.FileCount.Opts;
 import org.apache.hadoop.io.Text;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
 
-public class CountTest extends TestCase {
+public class CountTest {
 
-  private static final Logger log = LoggerFactory.getLogger(CountTest.class);
+  @Rule
+  public TestName test = new TestName();
 
-  {
-    try {
-      Connector conn = new MockInstance("counttest").getConnector("root", new PasswordToken(""));
-      conn.tableOperations().create("dirlisttable");
-      BatchWriter bw = conn.createBatchWriter("dirlisttable", new BatchWriterConfig());
-      ColumnVisibility cv = new ColumnVisibility();
-      // / has 1 dir
-      // /local has 2 dirs 1 file
-      // /local/user1 has 2 files
-      bw.addMutation(Ingest.buildMutation(cv, "/local", true, false, true, 272, 12345, null));
-      bw.addMutation(Ingest.buildMutation(cv, "/local/user1", true, false, true, 272, 12345, null));
-      bw.addMutation(Ingest.buildMutation(cv, "/local/user2", true, false, true, 272, 12345, null));
-      bw.addMutation(Ingest.buildMutation(cv, "/local/file", false, false, false, 1024, 12345, null));
-      bw.addMutation(Ingest.buildMutation(cv, "/local/file", false, false, false, 1024, 23456, null));
-      bw.addMutation(Ingest.buildMutation(cv, "/local/user1/file1", false, false, false, 2024, 12345, null));
-      bw.addMutation(Ingest.buildMutation(cv, "/local/user1/file2", false, false, false, 1028, 23456, null));
-      bw.close();
-    } catch (Exception e) {
-      log.error("Could not add mutations in initializer.", e);
-    }
+  private Connector conn;
+  private String tableName;
+
+  @Before
+  public void setupInstance() throws Exception {
+    tableName = test.getMethodName();
+    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
+    conn = inst.getConnector("root", new PasswordToken(""));
+    conn.tableOperations().create(tableName);
+    BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig());
+    ColumnVisibility cv = new ColumnVisibility();
+    // / has 1 dir
+    // /local has 2 dirs 1 file
+    // /local/user1 has 2 files
+    bw.addMutation(Ingest.buildMutation(cv, "/local", true, false, true, 272, 12345, null));
+    bw.addMutation(Ingest.buildMutation(cv, "/local/user1", true, false, true, 272, 12345, null));
+    bw.addMutation(Ingest.buildMutation(cv, "/local/user2", true, false, true, 272, 12345, null));
+    bw.addMutation(Ingest.buildMutation(cv, "/local/file", false, false, false, 1024, 12345, null));
+    bw.addMutation(Ingest.buildMutation(cv, "/local/file", false, false, false, 1024, 23456, null));
+    bw.addMutation(Ingest.buildMutation(cv, "/local/user1/file1", false, false, false, 2024, 12345, null));
+    bw.addMutation(Ingest.buildMutation(cv, "/local/user1/file2", false, false, false, 1028, 23456, null));
+    bw.close();
   }
 
+  @Test
   public void test() throws Exception {
-    Scanner scanner = new MockInstance("counttest").getConnector("root", new PasswordToken("")).createScanner("dirlisttable", new Authorizations());
+    Scanner scanner = conn.createScanner(tableName, new Authorizations());
     scanner.fetchColumn(new Text("dir"), new Text("counts"));
     assertFalse(scanner.iterator().hasNext());
 
@@ -75,7 +82,7 @@ public class CountTest extends TestCase {
     ScannerOpts scanOpts = new ScannerOpts();
     BatchWriterOpts bwOpts = new BatchWriterOpts();
     opts.instance = "counttest";
-    opts.setTableName("dirlisttable");
+    opts.setTableName(tableName);
     opts.setPassword(new Password("secret"));
     opts.mock = true;
     opts.setPassword(new Opts.Password(""));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
----------------------------------------------------------------------
diff --git a/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java b/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
index b95c00c..997612f 100644
--- a/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
+++ b/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
@@ -33,7 +33,7 @@ import java.util.Map.Entry;
 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.mock.MockInstance;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -47,8 +47,11 @@ import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TestName;
 
 public class ChunkInputFormatTest {
 
@@ -62,6 +65,17 @@ public class ChunkInputFormatTest {
   private static List<Entry<Key,Value>> data;
   private static List<Entry<Key,Value>> baddata;
 
+  private Connector conn;
+
+  @Rule
+  public TestName test = new TestName();
+
+  @Before
+  public void setupInstance() throws Exception {
+    Instance instance = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
+    conn = instance.getConnector("root", new PasswordToken(""));
+  }
+
   @BeforeClass
   public static void setupClass() {
     System.setProperty("hadoop.tmp.dir", System.getProperty("user.dir") + "/target/hadoop-tmp");
@@ -243,8 +257,6 @@ public class ChunkInputFormatTest {
 
   @Test
   public void test() throws Exception {
-    MockInstance instance = new MockInstance("instance1");
-    Connector conn = instance.getConnector("root", new PasswordToken(""));
     conn.tableOperations().create("test");
     BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig());
 
@@ -256,15 +268,13 @@ public class ChunkInputFormatTest {
     }
     bw.close();
 
-    assertEquals(0, CIFTester.main("instance1", "root", "", "test", CIFTester.TestMapper.class.getName()));
+    assertEquals(0, CIFTester.main(conn.getInstance().getInstanceName(), "root", "", "test", CIFTester.TestMapper.class.getName()));
     assertNull(e1);
     assertNull(e2);
   }
 
   @Test
   public void testErrorOnNextWithoutClose() throws Exception {
-    MockInstance instance = new MockInstance("instance2");
-    Connector conn = instance.getConnector("root", new PasswordToken(""));
     conn.tableOperations().create("test");
     BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig());
 
@@ -276,7 +286,7 @@ public class ChunkInputFormatTest {
     }
     bw.close();
 
-    assertEquals(1, CIFTester.main("instance2", "root", "", "test", CIFTester.TestNoClose.class.getName()));
+    assertEquals(1, CIFTester.main(conn.getInstance().getInstanceName(), "root", "", "test", CIFTester.TestNoClose.class.getName()));
     assertNull(e1);
     assertNull(e2);
     assertNotNull(e3);
@@ -284,8 +294,6 @@ public class ChunkInputFormatTest {
 
   @Test
   public void testInfoWithoutChunks() throws Exception {
-    MockInstance instance = new MockInstance("instance3");
-    Connector conn = instance.getConnector("root", new PasswordToken(""));
     conn.tableOperations().create("test");
     BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig());
     for (Entry<Key,Value> e : baddata) {
@@ -296,7 +304,7 @@ public class ChunkInputFormatTest {
     }
     bw.close();
 
-    assertEquals(0, CIFTester.main("instance3", "root", "", "test", CIFTester.TestBadData.class.getName()));
+    assertEquals(0, CIFTester.main(conn.getInstance().getInstanceName(), "root", "", "test", CIFTester.TestBadData.class.getName()));
     assertNull(e0);
     assertNull(e1);
     assertNull(e2);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputStreamTest.java
----------------------------------------------------------------------
diff --git a/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputStreamTest.java b/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputStreamTest.java
index 1976b21..614a480 100644
--- a/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputStreamTest.java
+++ b/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputStreamTest.java
@@ -16,22 +16,26 @@
  */
 package org.apache.accumulo.examples.simple.filedata;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map.Entry;
 
-import junit.framework.TestCase;
-
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
 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.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyValue;
@@ -41,17 +45,21 @@ import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.util.PeekingIterator;
 import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class ChunkInputStreamTest extends TestCase {
+public class ChunkInputStreamTest {
   private static final Logger log = LoggerFactory.getLogger(ChunkInputStream.class);
   List<Entry<Key,Value>> data;
   List<Entry<Key,Value>> baddata;
   List<Entry<Key,Value>> multidata;
 
-  {
+  @Before
+  public void setupData() {
     data = new ArrayList<Entry<Key,Value>>();
     addData(data, "a", "refs", "id\0ext", "A&B", "ext");
     addData(data, "a", "refs", "id\0name", "A&B", "name");
@@ -227,9 +235,13 @@ public class ChunkInputStreamTest extends TestCase {
     assertFalse(pi.hasNext());
   }
 
+  @Rule
+  public TestName test = new TestName();
+
   @Test
   public void testWithAccumulo() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException, IOException {
-    Connector conn = new MockInstance().getConnector("root", new PasswordToken(""));
+    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
+    Connector conn = inst.getConnector("root", new PasswordToken(""));
     conn.tableOperations().create("test");
     BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig());
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
index 87dfff8..ba52f80 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java
@@ -31,7 +31,7 @@ import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 
 /**
  * A utility class that will create Zookeeper and Accumulo processes that write all of their data to a single local directory. This class makes it easy to test
- * code against a real Accumulo instance. Its much more accurate for testing than {@link org.apache.accumulo.core.client.mock.MockAccumulo}, but much slower.
+ * code against a real Accumulo instance. The use of this utility will yield results which closely match a normal Accumulo instance.
  *
  * @since 1.5.0
  */

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/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 0cb14c7..772ba69 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
@@ -78,6 +78,7 @@ import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.master.state.SetGoalState;
+import org.apache.accumulo.minicluster.MiniAccumuloCluster;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.server.Accumulo;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -113,8 +114,8 @@ import com.google.common.collect.Maps;
 import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 /**
- * A utility class that will create Zookeeper and Accumulo processes that write all of their data to a single local directory. This class makes it easy to test
- * code against a real Accumulo instance. Its much more accurate for testing than {@link org.apache.accumulo.core.client.mock.MockAccumulo}, but much slower.
+ * This class provides the backing implementation for {@link MiniAccumuloCluster}, and may contain features for internal testing which have not yet been
+ * promoted to the public API. It's best to use {@link MiniAccumuloCluster} whenever possible. Use of this class risks API breakage between versions.
  *
  * @since 1.6.0
  */

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/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 88dad8d..cda04a8 100644
--- a/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
+++ b/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java
@@ -62,7 +62,6 @@ import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
@@ -80,6 +79,7 @@ import org.apache.accumulo.core.security.ColumnVisibility;
 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;
@@ -186,7 +186,7 @@ public class ProxyServer implements AccumuloProxy.Iface {
 
     String useMock = props.getProperty("useMockInstance");
     if (useMock != null && Boolean.parseBoolean(useMock))
-      instance = new MockInstance();
+      instance = DeprecationUtil.makeMockInstance(this.getClass().getName());
     else {
       ClientConfiguration clientConf;
       if (props.containsKey("clientConfigurationFile")) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/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 a35f39c..9198b19 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
@@ -26,11 +26,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.mock.MockInstance;
 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;
@@ -95,7 +95,7 @@ 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 (instance instanceof MockInstance) {
+    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/2556df3a/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 10cab49..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,7 +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.client.mock.MockInstance;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 
 public class ClientOnDefaultTable extends org.apache.accumulo.core.cli.ClientOnDefaultTable {
@@ -32,7 +32,7 @@ public class ClientOnDefaultTable extends org.apache.accumulo.core.cli.ClientOnD
       return cachedInstance;
 
     if (mock)
-      return cachedInstance = new MockInstance(instance);
+      return cachedInstance = DeprecationUtil.makeMockInstance(instance);
     if (instance == null) {
       return cachedInstance = HdfsZooInstance.getInstance();
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/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 e134235..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,7 +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.client.mock.MockInstance;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 
 public class ClientOnRequiredTable extends org.apache.accumulo.core.cli.ClientOnRequiredTable {
@@ -32,7 +32,7 @@ public class ClientOnRequiredTable extends org.apache.accumulo.core.cli.ClientOn
       return cachedInstance;
 
     if (mock)
-      return cachedInstance = new MockInstance(instance);
+      return cachedInstance = DeprecationUtil.makeMockInstance(instance);
     if (instance == null) {
       return cachedInstance = HdfsZooInstance.getInstance();
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/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 c50d95d..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,7 +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.client.mock.MockInstance;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 
 public class ClientOpts extends org.apache.accumulo.core.cli.ClientOpts {
@@ -30,7 +30,7 @@ public class ClientOpts extends org.apache.accumulo.core.cli.ClientOpts {
   @Override
   public Instance getInstance() {
     if (mock)
-      return new MockInstance(instance);
+      return DeprecationUtil.makeMockInstance(instance);
     if (instance == null) {
       return HdfsZooInstance.getInstance();
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/base/src/test/java/org/apache/accumulo/server/AccumuloServerContextTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/AccumuloServerContextTest.java b/server/base/src/test/java/org/apache/accumulo/server/AccumuloServerContextTest.java
index a596d9f..521fd7b 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/AccumuloServerContextTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/AccumuloServerContextTest.java
@@ -26,9 +26,9 @@ import java.util.Map.Entry;
 
 import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.Credentials;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -68,7 +68,7 @@ public class AccumuloServerContextTest {
     testUser.doAs(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
-        MockInstance instance = new MockInstance();
+        Instance instance = EasyMock.createMock(Instance.class);
 
         ClientConfiguration clientConf = ClientConfiguration.loadDefault();
         clientConf.setProperty(ClientProperty.INSTANCE_RPC_SASL_ENABLED, "true");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java b/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
index 8d649d2..fe4aff2 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/client/BulkImporterTest.java
@@ -25,15 +25,12 @@ import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.ClientContext;
-import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
-import org.apache.accumulo.core.client.mock.MockInstance;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
@@ -48,6 +45,7 @@ import org.apache.commons.lang.NotImplementedException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
+import org.easymock.EasyMock;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -109,7 +107,9 @@ public class BulkImporterTest {
   public void testFindOverlappingTablets() throws Exception {
     MockTabletLocator locator = new MockTabletLocator();
     FileSystem fs = FileSystem.getLocal(CachedConfiguration.getInstance());
-    ClientContext context = new ClientContext(new MockInstance(), new Credentials("root", new PasswordToken("")), new ClientConfiguration());
+    ClientContext context = EasyMock.createMock(ClientContext.class);
+    EasyMock.expect(context.getConfiguration()).andReturn(DefaultConfiguration.getInstance()).anyTimes();
+    EasyMock.replay(context);
     String file = "target/testFile.rf";
     fs.delete(new Path(file), true);
     FileSKVWriter writer = FileOperations.getInstance().openWriter(file, fs, fs.getConf(), context.getConfiguration());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
index c957009..7bf5d2d 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
@@ -26,9 +26,9 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -71,7 +71,7 @@ public class TableLoadBalancerTest {
     return result;
   }
 
-  static MockInstance instance = new MockInstance("mockamatic");
+  static Instance instance = new org.apache.accumulo.core.client.mock.MockInstance(TableLoadBalancerTest.class.getName());
 
   static SortedMap<TServerInstance,TabletServerStatus> state;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportingIteratorTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportingIteratorTest.java b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportingIteratorTest.java
index 2e0ad0c..76d0171 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportingIteratorTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportingIteratorTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.server.problems;
 
-import static org.easymock.EasyMock.createMock;
 import static org.easymock.EasyMock.expect;
 import static org.easymock.EasyMock.replay;
 import static org.easymock.EasyMock.verify;
@@ -28,14 +27,12 @@ import static org.junit.Assert.assertTrue;
 import java.util.Collection;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.accumulo.core.client.mock.MockInstance;
 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.InterruptibleIterator;
-import org.apache.accumulo.server.AccumuloServerContext;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
+import org.easymock.EasyMock;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -48,16 +45,15 @@ public class ProblemReportingIteratorTest {
 
   @Before
   public void setUp() throws Exception {
-    ii = createMock(InterruptibleIterator.class);
-    AccumuloServerContext context = new AccumuloServerContext(new ServerConfigurationFactory(new MockInstance()));
-    pri = new ProblemReportingIterator(context, TABLE, RESOURCE, false, ii);
+    ii = EasyMock.createMock(InterruptibleIterator.class);
+    pri = new ProblemReportingIterator(null, TABLE, RESOURCE, false, ii);
   }
 
   @Test
   public void testBasicGetters() {
-    Key key = createMock(Key.class);
+    Key key = EasyMock.createMock(Key.class);
     expect(ii.getTopKey()).andReturn(key);
-    Value value = createMock(Value.class);
+    Value value = EasyMock.createMock(Value.class);
     expect(ii.getTopValue()).andReturn(value);
     expect(ii.hasTop()).andReturn(true);
     replay(ii);
@@ -84,7 +80,7 @@ public class ProblemReportingIteratorTest {
 
   @Test
   public void testSeek() throws Exception {
-    Range r = createMock(Range.class);
+    Range r = EasyMock.createMock(Range.class);
     Collection<ByteSequence> f = new java.util.HashSet<ByteSequence>();
     ii.seek(r, f, true);
     replay(ii);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java b/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java
index 1af908b..81a3892 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java
@@ -23,24 +23,26 @@ import java.io.File;
 import java.io.IOException;
 import java.util.UUID;
 
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.ConnectorImpl;
 import org.apache.accumulo.core.client.impl.Credentials;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.security.SystemCredentials.SystemToken;
+import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TestName;
 
-/**
- *
- */
 public class SystemCredentialsTest {
 
-  private static MockInstance inst;
+  @Rule
+  public TestName test = new TestName();
+
+  private Instance inst;
 
   @BeforeClass
   public static void setUp() throws IOException {
-    inst = new MockInstance();
     File testInstanceId = new File(new File(new File(new File("target"), "instanceTest"), ServerConstants.INSTANCE_ID_DIR), UUID.fromString(
         "00000000-0000-0000-0000-000000000000").toString());
     if (!testInstanceId.exists()) {
@@ -55,6 +57,11 @@ public class SystemCredentialsTest {
     }
   }
 
+  @Before
+  public void setupInstance() {
+    inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
+  }
+
   /**
    * This is a test to ensure the string literal in {@link ConnectorImpl#ConnectorImpl(org.apache.accumulo.core.client.impl.ClientContext)} is kept up-to-date
    * if we move the {@link SystemToken}<br/>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/base/src/test/java/org/apache/accumulo/server/util/CloneTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/CloneTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/CloneTest.java
index 0c696a1..aa7cad4 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/CloneTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/CloneTest.java
@@ -16,16 +16,17 @@
  */
 package org.apache.accumulo.server.util;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
 import java.util.HashSet;
 import java.util.Map.Entry;
 
-import junit.framework.TestCase;
-
 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.Scanner;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -36,13 +37,26 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
 
-public class CloneTest extends TestCase {
+public class CloneTest {
 
-  public void testNoFiles() throws Exception {
-    MockInstance mi = new MockInstance();
-    Connector conn = mi.getConnector("", new PasswordToken(""));
+  private Connector conn;
+
+  @Rule
+  public TestName test = new TestName();
 
+  @Before
+  public void setupInstance() throws Exception {
+    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
+    conn = inst.getConnector("", new PasswordToken(""));
+  }
+
+  @Test
+  public void testNoFiles() throws Exception {
     KeyExtent ke = new KeyExtent(new Text("0"), null, null);
     Mutation mut = ke.getPrevRowUpdateMutation();
 
@@ -67,10 +81,8 @@ public class CloneTest extends TestCase {
 
   }
 
+  @Test
   public void testFilesChange() throws Exception {
-    MockInstance mi = new MockInstance();
-    Connector conn = mi.getConnector("", new PasswordToken(""));
-
     KeyExtent ke = new KeyExtent(new Text("0"), null, null);
     Mutation mut = ke.getPrevRowUpdateMutation();
 
@@ -119,10 +131,8 @@ public class CloneTest extends TestCase {
   }
 
   // test split where files of children are the same
+  @Test
   public void testSplit1() throws Exception {
-    MockInstance mi = new MockInstance();
-    Connector conn = mi.getConnector("", new PasswordToken(""));
-
     BatchWriter bw1 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
 
     bw1.addMutation(createTablet("0", null, null, "/default_tablet", "/default_tablet/0_0.rf"));
@@ -161,10 +171,8 @@ public class CloneTest extends TestCase {
   }
 
   // test split where files of children differ... like majc and split occurred
+  @Test
   public void testSplit2() throws Exception {
-    MockInstance mi = new MockInstance();
-    Connector conn = mi.getConnector("", new PasswordToken(""));
-
     BatchWriter bw1 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
 
     bw1.addMutation(createTablet("0", null, null, "/default_tablet", "/default_tablet/0_0.rf"));
@@ -232,10 +240,8 @@ public class CloneTest extends TestCase {
   }
 
   // test two tablets splitting into four
+  @Test
   public void testSplit3() throws Exception {
-    MockInstance mi = new MockInstance();
-    Connector conn = mi.getConnector("", new PasswordToken(""));
-
     BatchWriter bw1 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
 
     bw1.addMutation(createTablet("0", "m", null, "/d1", "/d1/file1"));
@@ -278,11 +284,8 @@ public class CloneTest extends TestCase {
   }
 
   // test cloned marker
+  @Test
   public void testClonedMarker() throws Exception {
-
-    MockInstance mi = new MockInstance();
-    Connector conn = mi.getConnector("", new PasswordToken(""));
-
     BatchWriter bw1 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
 
     bw1.addMutation(createTablet("0", "m", null, "/d1", "/d1/file1"));
@@ -343,10 +346,8 @@ public class CloneTest extends TestCase {
   }
 
   // test two tablets splitting into four
+  @Test
   public void testMerge() throws Exception {
-    MockInstance mi = new MockInstance();
-    Connector conn = mi.getConnector("", new PasswordToken(""));
-
     BatchWriter bw1 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
 
     bw1.addMutation(createTablet("0", "m", null, "/d1", "/d1/file1"));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
index 04a83d3..59720e9 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
@@ -31,7 +31,6 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.UUID;
 
-import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.IteratorSetting.Column;
@@ -39,7 +38,6 @@ import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.impl.Writer;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.ColumnUpdate;
@@ -81,7 +79,9 @@ public class ReplicationTableUtilTest {
     EasyMock.replay(writer);
 
     Credentials creds = new Credentials("root", new PasswordToken(""));
-    ClientContext context = new ClientContext(new MockInstance(), creds, new ClientConfiguration());
+    ClientContext context = EasyMock.createMock(ClientContext.class);
+    EasyMock.expect(context.getCredentials()).andReturn(creds).anyTimes();
+    EasyMock.replay(context);
 
     // Magic hook to create a Writer
     ReplicationTableUtil.addWriter(creds, writer);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/base/src/test/java/org/apache/accumulo/server/util/TabletIteratorTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/TabletIteratorTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/TabletIteratorTest.java
index 9c6cee1..19bf13a 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/TabletIteratorTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/TabletIteratorTest.java
@@ -18,13 +18,11 @@ package org.apache.accumulo.server.util;
 
 import java.util.Map.Entry;
 
-import junit.framework.TestCase;
-
 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.Scanner;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -37,8 +35,18 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.server.util.TabletIterator.TabletDeletedException;
 import org.apache.hadoop.io.Text;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TestName;
+
+public class TabletIteratorTest {
+
+  @Rule
+  public TestName test = new TestName();
 
-public class TabletIteratorTest extends TestCase {
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
 
   class TestTabletIterator extends TabletIterator {
 
@@ -78,8 +86,9 @@ public class TabletIteratorTest extends TestCase {
   }
 
   // simulate a merge happening while iterating over tablets
+  @Test
   public void testMerge() throws Exception {
-    MockInstance mi = new MockInstance();
+    Instance mi = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
     Connector conn = mi.getConnector("", new PasswordToken(""));
 
     KeyExtent ke1 = new KeyExtent(new Text("0"), new Text("m"), null);
@@ -97,11 +106,9 @@ public class TabletIteratorTest extends TestCase {
 
     TestTabletIterator tabIter = new TestTabletIterator(conn);
 
-    try {
-      while (tabIter.hasNext()) {
-        tabIter.next();
-      }
-      assertTrue(false);
-    } catch (TabletDeletedException tde) {}
+    exception.expect(TabletDeletedException.class);
+    while (tabIter.hasNext()) {
+      tabIter.next();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
----------------------------------------------------------------------
diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
index 7460b66..92a72fb 100644
--- a/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
@@ -31,7 +31,6 @@ 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.Scanner;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigurationCopy;
@@ -64,22 +63,28 @@ import com.google.common.collect.Iterables;
 public class CloseWriteAheadLogReferencesTest {
 
   private CloseWriteAheadLogReferences refs;
-  private Instance inst;
+  private Connector conn;
 
   @Rule
   public TestName testName = new TestName();
 
   @Before
+  public void setupInstance() throws Exception {
+    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(testName.getMethodName());
+    conn = inst.getConnector("root", new PasswordToken(""));
+  }
+
+  @Before
   public void setup() {
-    inst = createMock(Instance.class);
+    Instance mockInst = createMock(Instance.class);
     SiteConfiguration siteConfig = EasyMock.createMock(SiteConfiguration.class);
-    expect(inst.getInstanceID()).andReturn(testName.getMethodName()).anyTimes();
-    expect(inst.getZooKeepers()).andReturn("localhost").anyTimes();
-    expect(inst.getZooKeepersSessionTimeOut()).andReturn(30000).anyTimes();
+    expect(mockInst.getInstanceID()).andReturn(testName.getMethodName()).anyTimes();
+    expect(mockInst.getZooKeepers()).andReturn("localhost").anyTimes();
+    expect(mockInst.getZooKeepersSessionTimeOut()).andReturn(30000).anyTimes();
     final AccumuloConfiguration systemConf = new ConfigurationCopy(new HashMap<String,String>());
     ServerConfigurationFactory factory = createMock(ServerConfigurationFactory.class);
     expect(factory.getConfiguration()).andReturn(systemConf).anyTimes();
-    expect(factory.getInstance()).andReturn(inst).anyTimes();
+    expect(factory.getInstance()).andReturn(mockInst).anyTimes();
     expect(factory.getSiteConfiguration()).andReturn(siteConfig).anyTimes();
 
     // Just make the SiteConfiguration delegate to our AccumuloConfiguration
@@ -106,16 +111,13 @@ public class CloseWriteAheadLogReferencesTest {
       }
     }).anyTimes();
 
-    replay(inst, factory, siteConfig);
+    replay(mockInst, factory, siteConfig);
     refs = new CloseWriteAheadLogReferences(new AccumuloServerContext(factory));
   }
 
   @Test
   public void unclosedWalsLeaveStatusOpen() throws Exception {
     Set<String> wals = Collections.emptySet();
-    Instance inst = new MockInstance(testName.getMethodName());
-    Connector conn = inst.getConnector("root", new PasswordToken(""));
-
     BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
     Mutation m = new Mutation(ReplicationSection.getRowPrefix() + "file:/accumulo/wal/tserver+port/12345");
     m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
@@ -134,9 +136,6 @@ public class CloseWriteAheadLogReferencesTest {
   public void closedWalsUpdateStatus() throws Exception {
     String file = "file:/accumulo/wal/tserver+port/12345";
     Set<String> wals = Collections.singleton(file);
-    Instance inst = new MockInstance(testName.getMethodName());
-    Connector conn = inst.getConnector("root", new PasswordToken(""));
-
     BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
     Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
     m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
@@ -155,9 +154,6 @@ public class CloseWriteAheadLogReferencesTest {
   public void partiallyReplicatedReferencedWalsAreNotClosed() throws Exception {
     String file = "file:/accumulo/wal/tserver+port/12345";
     Set<String> wals = Collections.singleton(file);
-    Instance inst = new MockInstance(testName.getMethodName());
-    Connector conn = inst.getConnector("root", new PasswordToken(""));
-
     BatchWriter bw = ReplicationTable.getBatchWriter(conn);
     Mutation m = new Mutation(file);
     StatusSection.add(m, new Text("1"), ProtobufUtil.toValue(StatusUtil.ingestedUntil(1000)));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java b/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java
index 3c3bc37..a18e5e9 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java
@@ -33,7 +33,6 @@ import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.impl.ReplicationOperationsImpl;
 import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -63,14 +62,14 @@ import org.slf4j.LoggerFactory;
 public class ReplicationOperationsImplTest {
   private static final Logger log = LoggerFactory.getLogger(ReplicationOperationsImplTest.class);
 
-  private MockInstance inst;
+  private Instance inst;
 
   @Rule
   public TestName test = new TestName();
 
   @Before
   public void setup() {
-    inst = new MockInstance(test.getMethodName());
+    inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java b/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
index 8cbea68..a8fe771 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
@@ -27,7 +27,6 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
@@ -49,13 +48,12 @@ import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.master.state.TabletLocationState;
 import org.apache.hadoop.io.Text;
 import org.junit.Assert;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TestName;
 
 import com.google.common.net.HostAndPort;
 
-/**
- *
- */
 public class TestMergeState {
 
   class MockCurrentState implements CurrentState {
@@ -104,9 +102,12 @@ public class TestMergeState {
     bw.close();
   }
 
+  @Rule
+  public TestName test = new TestName();
+
   @Test
   public void test() throws Exception {
-    Instance instance = new MockInstance();
+    Instance instance = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
     AccumuloServerContext context = new AccumuloServerContext(new ServerConfigurationFactory(instance));
     Connector connector = context.getConnector();
     BatchWriter bw = connector.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/master/src/test/java/org/apache/accumulo/master/replication/FinishedWorkUpdaterTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/FinishedWorkUpdaterTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/FinishedWorkUpdaterTest.java
index 4f1e159..864a79d 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/FinishedWorkUpdaterTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/FinishedWorkUpdaterTest.java
@@ -21,8 +21,8 @@ import java.util.UUID;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -52,7 +52,7 @@ public class FinishedWorkUpdaterTest {
 
   @Before
   public void setup() throws Exception {
-    MockInstance inst = new MockInstance(test.getMethodName());
+    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
     conn = inst.getConnector("root", new PasswordToken(""));
     updater = new FinishedWorkUpdater(conn);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java
index 952fb2c..2555077 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java
@@ -25,8 +25,8 @@ import java.util.UUID;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.BatchWriter;
 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.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -54,7 +54,7 @@ import com.google.common.collect.Iterables;
 public class RemoveCompleteReplicationRecordsTest {
 
   private RemoveCompleteReplicationRecords rcrr;
-  private MockInstance inst;
+  private Instance inst;
   private Connector conn;
 
   @Rule
@@ -62,7 +62,7 @@ public class RemoveCompleteReplicationRecordsTest {
 
   @Before
   public void initialize() throws Exception {
-    inst = new MockInstance(test.getMethodName());
+    inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
     conn = inst.getConnector("root", new PasswordToken(""));
     rcrr = new RemoveCompleteReplicationRecords(conn);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
index e05a17e..d4675db 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
@@ -29,8 +29,6 @@ import java.util.TreeMap;
 import org.apache.accumulo.core.client.BatchWriter;
 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.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.Mutation;
@@ -60,13 +58,21 @@ public class SequentialWorkAssignerTest {
 
   private AccumuloConfiguration conf;
   private Connector conn;
+  private Connector mockConn;
   private SequentialWorkAssigner assigner;
 
   @Before
-  public void init() {
+  public void init() throws Exception {
     conf = createMock(AccumuloConfiguration.class);
     conn = createMock(Connector.class);
     assigner = new SequentialWorkAssigner(conf, conn);
+
+    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
+    mockConn = inst.getConnector("root", new PasswordToken(""));
+    // Set the connector
+    assigner.setConnector(mockConn);
+    // grant ourselves write to the replication table
+    mockConn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
   }
 
   @Test
@@ -74,18 +80,8 @@ public class SequentialWorkAssignerTest {
     ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
     Text serializedTarget = target.toText();
 
-    MockInstance inst = new MockInstance(test.getMethodName());
-    Credentials creds = new Credentials("root", new PasswordToken(""));
-    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
-
-    // Set the connector
-    assigner.setConnector(conn);
-
-    // grant ourselves write to the replication table
-    conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
-
     // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    BatchWriter bw = ReplicationTable.getBatchWriter(mockConn);
     // We want the name of file2 to sort before file1
     String filename1 = "z_file1", filename2 = "a_file1";
     String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
@@ -146,18 +142,8 @@ public class SequentialWorkAssignerTest {
     ReplicationTarget target2 = new ReplicationTarget("cluster1", "table2", "2");
     Text serializedTarget2 = target2.toText();
 
-    MockInstance inst = new MockInstance(test.getMethodName());
-    Credentials creds = new Credentials("root", new PasswordToken(""));
-    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
-
-    // Set the connector
-    assigner.setConnector(conn);
-
-    // grant ourselves write to the replication table
-    conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
-
     // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    BatchWriter bw = ReplicationTable.getBatchWriter(mockConn);
     // We want the name of file2 to sort before file1
     String filename1 = "z_file1", filename2 = "a_file1";
     String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
@@ -225,18 +211,8 @@ public class SequentialWorkAssignerTest {
     ReplicationTarget target2 = new ReplicationTarget("cluster2", "table1", "1");
     Text serializedTarget2 = target2.toText();
 
-    MockInstance inst = new MockInstance(test.getMethodName());
-    Credentials creds = new Credentials("root", new PasswordToken(""));
-    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
-
-    // Set the connector
-    assigner.setConnector(conn);
-
-    // grant ourselves write to the replication table
-    conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
-
     // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    BatchWriter bw = ReplicationTable.getBatchWriter(mockConn);
     // We want the name of file2 to sort before file1
     String filename1 = "z_file1", filename2 = "a_file1";
     String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
@@ -345,18 +321,8 @@ public class SequentialWorkAssignerTest {
     ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
     Text serializedTarget = target.toText();
 
-    MockInstance inst = new MockInstance(test.getMethodName());
-    Credentials creds = new Credentials("root", new PasswordToken(""));
-    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
-
-    // Set the connector
-    assigner.setConnector(conn);
-
-    // grant ourselves write to the replication table
-    conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
-
     // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    BatchWriter bw = ReplicationTable.getBatchWriter(mockConn);
     // We want the name of file2 to sort before file1
     String filename1 = "z_file1", filename2 = "a_file1";
     String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java
index b57fd89..11be4fb 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java
@@ -28,9 +28,9 @@ import java.util.UUID;
 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.Scanner;
 import org.apache.accumulo.core.client.impl.Credentials;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -46,6 +46,7 @@ import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.apache.accumulo.server.util.ReplicationTableUtil;
 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;
@@ -58,12 +59,17 @@ public class StatusMakerTest {
   @Rule
   public TestName test = new TestName();
 
-  @Test
-  public void statusRecordsCreated() throws Exception {
-    MockInstance inst = new MockInstance(test.getMethodName());
+  private Connector conn;
+
+  @Before
+  public void setupInstance() throws Exception {
+    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
     Credentials creds = new Credentials("root", new PasswordToken(""));
-    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
+    conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
+  }
 
+  @Test
+  public void statusRecordsCreated() throws Exception {
     String sourceTable = "source";
     conn.tableOperations().create(sourceTable);
     ReplicationTableUtil.configureMetadataTable(conn, sourceTable);
@@ -111,10 +117,6 @@ public class StatusMakerTest {
 
   @Test
   public void openMessagesAreNotDeleted() throws Exception {
-    MockInstance inst = new MockInstance(test.getMethodName());
-    Credentials creds = new Credentials("root", new PasswordToken(""));
-    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
-
     String sourceTable = "source";
     conn.tableOperations().create(sourceTable);
     ReplicationTableUtil.configureMetadataTable(conn, sourceTable);
@@ -151,10 +153,6 @@ public class StatusMakerTest {
 
   @Test
   public void closedMessagesAreDeleted() throws Exception {
-    MockInstance inst = new MockInstance(test.getMethodName());
-    Credentials creds = new Credentials("root", new PasswordToken(""));
-    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
-
     String sourceTable = "source";
     conn.tableOperations().create(sourceTable);
     ReplicationTableUtil.configureMetadataTable(conn, sourceTable);
@@ -198,10 +196,6 @@ public class StatusMakerTest {
 
   @Test
   public void closedMessagesCreateOrderRecords() throws Exception {
-    MockInstance inst = new MockInstance(test.getMethodName());
-    Credentials creds = new Credentials("root", new PasswordToken(""));
-    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
-
     String sourceTable = "source";
     conn.tableOperations().create(sourceTable);
     ReplicationTableUtil.configureMetadataTable(conn, sourceTable);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/master/src/test/java/org/apache/accumulo/master/replication/UnorderedWorkAssignerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/UnorderedWorkAssignerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/UnorderedWorkAssignerTest.java
index 35df344..f4c53f0 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/UnorderedWorkAssignerTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/UnorderedWorkAssignerTest.java
@@ -33,8 +33,6 @@ import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
 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.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.Mutation;
@@ -65,13 +63,17 @@ public class UnorderedWorkAssignerTest {
 
   private AccumuloConfiguration conf;
   private Connector conn;
+  private Connector mockConn;
   private UnorderedWorkAssigner assigner;
 
   @Before
-  public void init() {
+  public void init() throws Exception {
     conf = createMock(AccumuloConfiguration.class);
     conn = createMock(Connector.class);
     assigner = new UnorderedWorkAssigner(conf, conn);
+
+    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
+    mockConn = inst.getConnector("root", new PasswordToken(""));
   }
 
   @Test
@@ -128,15 +130,11 @@ public class UnorderedWorkAssignerTest {
         + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target2.getRemoteIdentifier() + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR
         + target2.getSourceTableId();
 
-    MockInstance inst = new MockInstance(test.getMethodName());
-    Credentials creds = new Credentials("root", new PasswordToken(""));
-    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
-
     // Set the connector
-    assigner.setConnector(conn);
+    assigner.setConnector(mockConn);
 
     // grant ourselves write to the replication table
-    conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
+    mockConn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
 
     Status.Builder builder = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(false).setCreatedTime(5l);
     Status status1 = builder.build();
@@ -144,7 +142,7 @@ public class UnorderedWorkAssignerTest {
     Status status2 = builder.build();
 
     // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    BatchWriter bw = ReplicationTable.getBatchWriter(mockConn);
     String filename1 = UUID.randomUUID().toString(), filename2 = UUID.randomUUID().toString();
     String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
     Mutation m = new Mutation(file1);
@@ -190,18 +188,14 @@ public class UnorderedWorkAssignerTest {
     ReplicationTarget target1 = new ReplicationTarget("cluster1", "table1", "1"), target2 = new ReplicationTarget("cluster1", "table2", "2");
     Text serializedTarget1 = target1.toText(), serializedTarget2 = target2.toText();
 
-    MockInstance inst = new MockInstance(test.getMethodName());
-    Credentials creds = new Credentials("root", new PasswordToken(""));
-    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
-
     // Set the connector
-    assigner.setConnector(conn);
+    assigner.setConnector(mockConn);
 
     // grant ourselves write to the replication table
-    conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
+    mockConn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
 
     // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    BatchWriter bw = ReplicationTable.getBatchWriter(mockConn);
     String filename1 = UUID.randomUUID().toString(), filename2 = UUID.randomUUID().toString();
     String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
 
@@ -261,18 +255,14 @@ public class UnorderedWorkAssignerTest {
 
     queuedWork.add("wal1|" + serializedTarget.toString());
 
-    MockInstance inst = new MockInstance(test.getMethodName());
-    Credentials creds = new Credentials("root", new PasswordToken(""));
-    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
-
     // Set the connector
-    assigner.setConnector(conn);
+    assigner.setConnector(mockConn);
 
     // grant ourselves write to the replication table
-    conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
+    mockConn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
 
     // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    BatchWriter bw = ReplicationTable.getBatchWriter(mockConn);
     String file1 = "/accumulo/wal/tserver+port/wal1";
     Mutation m = new Mutation(file1);
     WorkSection.add(m, target.toText(), StatusUtil.openWithUnknownLengthValue());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java
index 6373def..d6d0b9a 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java
@@ -23,8 +23,8 @@ import java.util.Set;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -51,7 +51,7 @@ import com.google.common.collect.Iterables;
 
 public class WorkMakerTest {
 
-  private MockInstance instance;
+  private Instance instance;
   private Connector conn;
 
   @Rule
@@ -60,7 +60,7 @@ public class WorkMakerTest {
 
   @Before
   public void createMockAccumulo() throws Exception {
-    instance = new MockInstance();
+    instance = new org.apache.accumulo.core.client.mock.MockInstance(name.getMethodName());
     context = new AccumuloServerContext(new ServerConfigurationFactory(instance));
     conn = context.getConnector();
     conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/server/tserver/src/test/java/org/apache/accumulo/tserver/LargestFirstMemoryManagerTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/LargestFirstMemoryManagerTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/LargestFirstMemoryManagerTest.java
index f3bd220..e0dc0c0 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/LargestFirstMemoryManagerTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/LargestFirstMemoryManagerTest.java
@@ -22,7 +22,6 @@ import java.util.Arrays;
 import java.util.List;
 
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.data.impl.KeyExtent;
@@ -34,6 +33,8 @@ import org.apache.accumulo.server.tabletserver.LargestFirstMemoryManager;
 import org.apache.accumulo.server.tabletserver.MemoryManagementActions;
 import org.apache.accumulo.server.tabletserver.TabletState;
 import org.apache.hadoop.io.Text;
+import org.easymock.EasyMock;
+import org.junit.Before;
 import org.junit.Test;
 
 import com.google.common.base.Function;
@@ -47,11 +48,18 @@ public class LargestFirstMemoryManagerTest {
   private static final long QGIG = ONE_GIG / 4;
   private static final long ONE_MINUTE = 60 * 1000;
 
+  private Instance inst;
+
+  @Before
+  public void mockInstance() {
+    inst = EasyMock.createMock(Instance.class);
+  }
+
   @Test
   public void test() throws Exception {
     LargestFirstMemoryManagerUnderTest mgr = new LargestFirstMemoryManagerUnderTest();
     ServerConfiguration config = new ServerConfiguration() {
-      ServerConfigurationFactory delegate = new ServerConfigurationFactory(new MockInstance());
+      ServerConfigurationFactory delegate = new ServerConfigurationFactory(inst);
 
       @Override
       public AccumuloConfiguration getConfiguration() {
@@ -176,7 +184,7 @@ public class LargestFirstMemoryManagerTest {
     };
     LargestFirstMemoryManagerWithExistenceCheck mgr = new LargestFirstMemoryManagerWithExistenceCheck(existenceCheck);
     ServerConfiguration config = new ServerConfiguration() {
-      ServerConfigurationFactory delegate = new ServerConfigurationFactory(new MockInstance());
+      ServerConfigurationFactory delegate = new ServerConfigurationFactory(inst);
 
       @Override
       public AccumuloConfiguration getConfiguration() {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/shell/src/main/java/org/apache/accumulo/shell/Shell.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/accumulo/shell/Shell.java b/shell/src/main/java/org/apache/accumulo/shell/Shell.java
index 4734cb1..b140fe7 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/Shell.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/Shell.java
@@ -59,7 +59,6 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
@@ -71,6 +70,7 @@ import org.apache.accumulo.core.data.thrift.TConstraintViolationSummary;
 import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 import org.apache.accumulo.core.trace.DistributedTrace;
 import org.apache.accumulo.core.util.BadArgumentException;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.core.util.format.BinaryFormatter;
 import org.apache.accumulo.core.util.format.DefaultFormatter;
 import org.apache.accumulo.core.util.format.Formatter;
@@ -315,7 +315,7 @@ public class Shell extends ShellOptions implements KeywordExecutable {
 
     tabCompletion = !options.isTabCompletionDisabled();
 
-    // Use a fake (Mock), ZK, or HdfsZK Accumulo instance
+    // Use a ZK, or HdfsZK Accumulo instance
     setInstance(options);
 
     // AuthenticationToken options
@@ -448,7 +448,7 @@ public class Shell extends ShellOptions implements KeywordExecutable {
     // should only be one set of instance options set
     instance = null;
     if (options.isFake()) {
-      instance = new MockInstance("fake");
+      instance = DeprecationUtil.makeMockInstance("fake");
     } else {
       String instanceName, hosts;
       if (options.isHdfsZooInstance()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/shell/src/main/java/org/apache/accumulo/shell/mock/MockShell.java
----------------------------------------------------------------------
diff --git a/shell/src/main/java/org/apache/accumulo/shell/mock/MockShell.java b/shell/src/main/java/org/apache/accumulo/shell/mock/MockShell.java
index 609f23f..f2bd075 100644
--- a/shell/src/main/java/org/apache/accumulo/shell/mock/MockShell.java
+++ b/shell/src/main/java/org/apache/accumulo/shell/mock/MockShell.java
@@ -28,7 +28,6 @@ import jline.console.ConsoleReader;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.shell.Shell;
 import org.apache.accumulo.shell.ShellOptionsJC;
 import org.apache.commons.cli.CommandLine;
@@ -36,7 +35,10 @@ import org.apache.commons.vfs2.FileSystemException;
 
 /**
  * An Accumulo Shell implementation that allows a developer to attach an InputStream and Writer to the Shell for testing purposes.
+ *
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
  */
+@Deprecated
 public class MockShell extends Shell {
   private static final String NEWLINE = "\n";
 
@@ -76,7 +78,7 @@ public class MockShell extends Shell {
   @Override
   protected void setInstance(ShellOptionsJC options) {
     // We always want a MockInstance for this test
-    instance = new MockInstance();
+    instance = new org.apache.accumulo.core.client.mock.MockInstance();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/shell/src/test/java/org/apache/accumulo/shell/ShellSetInstanceTest.java
----------------------------------------------------------------------
diff --git a/shell/src/test/java/org/apache/accumulo/shell/ShellSetInstanceTest.java b/shell/src/test/java/org/apache/accumulo/shell/ShellSetInstanceTest.java
index e592abb..4d74f65 100644
--- a/shell/src/test/java/org/apache/accumulo/shell/ShellSetInstanceTest.java
+++ b/shell/src/test/java/org/apache/accumulo/shell/ShellSetInstanceTest.java
@@ -39,7 +39,6 @@ import jline.console.ConsoleReader;
 import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.ConfigSanityCheck;
 import org.apache.accumulo.core.conf.Property;
@@ -124,12 +123,12 @@ public class ShellSetInstanceTest {
     ShellOptionsJC opts = createMock(ShellOptionsJC.class);
     expect(opts.isFake()).andReturn(true);
     replay(opts);
-    MockInstance theInstance = createMock(MockInstance.class);
-    expectNew(MockInstance.class, "fake").andReturn(theInstance);
-    replay(theInstance, MockInstance.class);
+    org.apache.accumulo.core.client.mock.MockInstance theInstance = createMock(org.apache.accumulo.core.client.mock.MockInstance.class);
+    expectNew(org.apache.accumulo.core.client.mock.MockInstance.class, "fake").andReturn(theInstance);
+    replay(theInstance, org.apache.accumulo.core.client.mock.MockInstance.class);
 
     shell.setInstance(opts);
-    verify(theInstance, MockInstance.class);
+    verify(theInstance, org.apache.accumulo.core.client.mock.MockInstance.class);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/shell/src/test/java/org/apache/accumulo/shell/commands/FormatterCommandTest.java
----------------------------------------------------------------------
diff --git a/shell/src/test/java/org/apache/accumulo/shell/commands/FormatterCommandTest.java b/shell/src/test/java/org/apache/accumulo/shell/commands/FormatterCommandTest.java
index 704d0c3..02a9ee2 100644
--- a/shell/src/test/java/org/apache/accumulo/shell/commands/FormatterCommandTest.java
+++ b/shell/src/test/java/org/apache/accumulo/shell/commands/FormatterCommandTest.java
@@ -33,14 +33,13 @@ import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.util.format.Formatter;
 import org.apache.accumulo.shell.Shell;
-import org.apache.accumulo.shell.mock.MockShell;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.junit.Assert;
 import org.junit.Test;
 
 /**
- * Uses the MockShell to test the shell output with Formatters
+ * Tests the shell output with Formatters
  */
 public class FormatterCommandTest {
   ByteArrayOutputStream out = null;
@@ -57,10 +56,10 @@ public class FormatterCommandTest {
 
     final String[] commands = createCommands();
 
-    in = MockShell.makeCommands(commands);
+    in = org.apache.accumulo.shell.mock.MockShell.makeCommands(commands);
     out = new ByteArrayOutputStream();
 
-    final MockShell shell = new MockShell(in, out);
+    final Shell shell = new org.apache.accumulo.shell.mock.MockShell(in, out);
     assertTrue("Failed to configure shell without error", shell.config(args));
 
     // Can't call createtable in the shell with MockAccumulo

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java b/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java
index 82c89be..271822d 100644
--- a/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java
+++ b/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java
@@ -28,7 +28,6 @@ import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -37,15 +36,16 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class RegExTest {
 
-  Instance inst = new MockInstance();
-  Connector conn;
+  private static Connector conn;
 
-  @Test
-  public void runTest() throws Exception {
+  @BeforeClass
+  public static void setupTests() throws Exception {
+    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(RegExTest.class.getName());
     conn = inst.getConnector("user", new PasswordToken("pass"));
     conn.tableOperations().create("ret");
     BatchWriter bw = conn.createBatchWriter("ret", new BatchWriterConfig());
@@ -71,12 +71,6 @@ public class RegExTest {
     }
 
     bw.close();
-
-    runTest1();
-    runTest2();
-    runTest3();
-    runTest4();
-    runTest5();
   }
 
   private void check(String regex, String val) throws Exception {
@@ -92,31 +86,36 @@ public class RegExTest {
     check(regex, val.toString());
   }
 
-  private void runTest1() throws Exception {
+  @Test
+  public void runTest1() throws Exception {
     // try setting all regex
     Range range = new Range(new Text("rf"), true, new Text("rl"), true);
     runTest(range, "r[g-k]", "cf[1-5]", "cq[x-z]", "v[g-k][1-5][t-y]", 5 * 5 * (3 - 1));
   }
 
-  private void runTest2() throws Exception {
+  @Test
+  public void runTest2() throws Exception {
     // try setting only a row regex
     Range range = new Range(new Text("rf"), true, new Text("rl"), true);
     runTest(range, "r[g-k]", null, null, null, 5 * 36 * 36);
   }
 
-  private void runTest3() throws Exception {
+  @Test
+  public void runTest3() throws Exception {
     // try setting only a col fam regex
     Range range = new Range((Key) null, (Key) null);
     runTest(range, null, "cf[a-f]", null, null, 36 * 6 * 36);
   }
 
-  private void runTest4() throws Exception {
+  @Test
+  public void runTest4() throws Exception {
     // try setting only a col qual regex
     Range range = new Range((Key) null, (Key) null);
     runTest(range, null, null, "cq[1-7]", null, 36 * 36 * 7);
   }
 
-  private void runTest5() throws Exception {
+  @Test
+  public void runTest5() throws Exception {
     // try setting only a value regex
     Range range = new Range((Key) null, (Key) null);
     runTest(range, null, null, null, "v[a-c][d-f][g-i]", 3 * 3 * 3);


[07/14] accumulo git commit: ACCUMULO-3920 Convert more tests from mock

Posted by ct...@apache.org.
ACCUMULO-3920 Convert more tests from mock

* Convert tests using mock to MiniAccumuloCluster
* Separate ITs from UTs, when possible
* Fix checkstyle/findbugs issues


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

Branch: refs/heads/master
Commit: cc3c0111ae2f96b9b48efc79500b6846f232b314
Parents: 66c62e5
Author: Christopher Tubbs <ct...@apache.org>
Authored: Fri Jul 17 17:46:13 2015 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Thu Jul 30 16:33:03 2015 -0400

----------------------------------------------------------------------
 .../core/client/ClientSideIteratorTest.java     | 140 ------
 .../mapred/AccumuloFileOutputFormatTest.java    | 183 -------
 .../client/mapred/AccumuloInputFormatTest.java  | 179 -------
 .../AccumuloMultiTableInputFormatTest.java      | 145 +-----
 .../client/mapred/AccumuloOutputFormatTest.java | 140 +-----
 .../mapred/AccumuloRowInputFormatTest.java      | 214 ---------
 .../core/client/mapred/TokenFileTest.java       | 179 -------
 .../mapreduce/AccumuloFileOutputFormatTest.java | 171 -------
 .../mapreduce/AccumuloInputFormatTest.java      | 246 ----------
 .../AccumuloMultiTableInputFormatTest.java      | 143 +-----
 .../mapreduce/AccumuloOutputFormatTest.java     | 127 -----
 .../mapreduce/AccumuloRowInputFormatTest.java   | 204 --------
 .../BadPasswordSplitsAccumuloInputFormat.java   |  42 --
 .../EmptySplitsAccumuloInputFormat.java         |  45 --
 .../core/client/mapreduce/TokenFileTest.java    | 171 -------
 .../user/IntersectingIteratorTest.java          |  13 -
 .../examples/simple/dirlist/FileCount.java      |   2 +-
 .../examples/simple/dirlist/CountTest.java      | 106 ----
 .../simple/filedata/ChunkInputFormatTest.java   | 312 ------------
 .../simple/filedata/ChunkInputStreamTest.java   |  82 +---
 .../server/util/TabletIteratorTest.java         | 114 -----
 .../CloseWriteAheadLogReferencesTest.java       | 171 -------
 .../master/MasterClientServiceHandler.java      |   7 +-
 .../master/ReplicationOperationsImplTest.java   | 451 -----------------
 .../apache/accumulo/master/TestMergeState.java  | 206 --------
 .../replication/FinishedWorkUpdaterTest.java    | 165 -------
 .../RemoveCompleteReplicationRecordsTest.java   | 327 -------------
 .../replication/SequentialWorkAssignerTest.java | 292 +----------
 .../master/replication/StatusMakerTest.java     | 253 ----------
 .../replication/UnorderedWorkAssignerTest.java  | 160 +------
 .../master/replication/WorkMakerTest.java       | 171 +------
 .../shell/commands/FormatterCommandTest.java    | 189 --------
 .../accumulo/test/AccumuloOutputFormatIT.java   | 100 ----
 .../accumulo/test/ClientSideIteratorIT.java     | 144 ++++++
 .../org/apache/accumulo/test/ShellServerIT.java | 112 +++++
 .../test/examples/simple/dirlist/CountIT.java   | 101 ++++
 .../simple/filedata/ChunkInputFormatIT.java     | 319 ++++++++++++
 .../simple/filedata/ChunkInputStreamIT.java     | 174 +++++++
 .../test/functional/AccumuloInputFormatIT.java  | 212 --------
 .../accumulo/test/functional/MergeIT.java       |  78 +++
 .../CloseWriteAheadLogReferencesIT.java         | 184 +++++++
 .../test/mapred/AccumuloFileOutputFormatIT.java | 204 ++++++++
 .../test/mapred/AccumuloInputFormatIT.java      | 195 ++++++++
 .../mapred/AccumuloMultiTableInputFormatIT.java | 163 +++++++
 .../test/mapred/AccumuloOutputFormatIT.java     | 227 +++++++++
 .../test/mapred/AccumuloRowInputFormatIT.java   | 202 ++++++++
 .../accumulo/test/mapred/TokenFileIT.java       | 174 +++++++
 .../mapreduce/AccumuloFileOutputFormatIT.java   | 211 ++++++++
 .../test/mapreduce/AccumuloInputFormatIT.java   | 479 +++++++++++++++++++
 .../AccumuloMultiTableInputFormatIT.java        | 161 +++++++
 .../test/mapreduce/AccumuloOutputFormatIT.java  | 153 ++++++
 .../mapreduce/AccumuloRowInputFormatIT.java     | 197 ++++++++
 .../accumulo/test/mapreduce/TokenFileIT.java    | 167 +++++++
 .../accumulo/test/master/MergeStateIT.java      | 204 ++++++++
 .../test/replication/FinishedWorkUpdaterIT.java | 173 +++++++
 .../RemoveCompleteReplicationRecordsIT.java     | 337 +++++++++++++
 .../ReplicationOperationsImplIT.java            | 447 +++++++++++++++++
 .../replication/SequentialWorkAssignerIT.java   | 368 ++++++++++++++
 .../test/replication/StatusMakerIT.java         | 249 ++++++++++
 .../replication/UnorderedWorkAssignerIT.java    | 238 +++++++++
 .../accumulo/test/replication/WorkMakerIT.java  | 209 ++++++++
 61 files changed, 5906 insertions(+), 5626 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java
deleted file mode 100644
index e0ce90e..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.client;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map.Entry;
-
-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.PartialKey;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.iterators.user.IntersectingIterator;
-import org.apache.accumulo.core.iterators.user.VersioningIterator;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.hadoop.io.Text;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-public class ClientSideIteratorTest {
-  List<Key> resultSet1;
-  List<Key> resultSet2;
-  List<Key> resultSet3;
-  {
-    resultSet1 = new ArrayList<Key>();
-    resultSet1.add(new Key("row1", "colf", "colq", 4l));
-    resultSet1.add(new Key("row1", "colf", "colq", 3l));
-    resultSet2 = new ArrayList<Key>();
-    resultSet2.add(new Key("row1", "colf", "colq", 4l));
-    resultSet2.add(new Key("row1", "colf", "colq", 3l));
-    resultSet2.add(new Key("row1", "colf", "colq", 2l));
-    resultSet2.add(new Key("row1", "colf", "colq", 1l));
-    resultSet3 = new ArrayList<Key>();
-    resultSet3.add(new Key("part1", "", "doc2"));
-    resultSet3.add(new Key("part2", "", "DOC2"));
-  }
-
-  private void checkResults(final Iterable<Entry<Key,Value>> scanner, final List<Key> results, final PartialKey pk) {
-    int i = 0;
-    for (Entry<Key,Value> entry : scanner) {
-      assertTrue(entry.getKey().equals(results.get(i++), pk));
-    }
-    assertEquals(i, results.size());
-  }
-
-  @Rule
-  public TestName test = new TestName();
-
-  private Connector conn;
-
-  @Before
-  public void setupInstance() throws Exception {
-    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    conn = inst.getConnector("root", new PasswordToken(""));
-  }
-
-  @Test
-  public void testIntersect() throws Exception {
-    conn.tableOperations().create("intersect");
-    BatchWriter bw = conn.createBatchWriter("intersect", new BatchWriterConfig());
-    Mutation m = new Mutation("part1");
-    m.put("bar", "doc1", "value");
-    m.put("bar", "doc2", "value");
-    m.put("dog", "doc3", "value");
-    m.put("foo", "doc2", "value");
-    m.put("foo", "doc3", "value");
-    bw.addMutation(m);
-    m = new Mutation("part2");
-    m.put("bar", "DOC1", "value");
-    m.put("bar", "DOC2", "value");
-    m.put("dog", "DOC3", "value");
-    m.put("foo", "DOC2", "value");
-    m.put("foo", "DOC3", "value");
-    bw.addMutation(m);
-    bw.flush();
-
-    final ClientSideIteratorScanner csis = new ClientSideIteratorScanner(conn.createScanner("intersect", new Authorizations()));
-    final IteratorSetting si = new IteratorSetting(10, "intersect", IntersectingIterator.class);
-    IntersectingIterator.setColumnFamilies(si, new Text[] {new Text("bar"), new Text("foo")});
-    csis.addScanIterator(si);
-
-    checkResults(csis, resultSet3, PartialKey.ROW_COLFAM_COLQUAL);
-  }
-
-  @Test
-  public void testVersioning() throws Exception {
-    conn.tableOperations().create("table");
-    conn.tableOperations().removeProperty("table", "table.iterator.scan.vers");
-    conn.tableOperations().removeProperty("table", "table.iterator.majc.vers");
-    conn.tableOperations().removeProperty("table", "table.iterator.minc.vers");
-    final BatchWriter bw = conn.createBatchWriter("table", new BatchWriterConfig());
-    Mutation m = new Mutation("row1");
-    m.put("colf", "colq", 1l, "value");
-    m.put("colf", "colq", 2l, "value");
-    bw.addMutation(m);
-    bw.flush();
-    m = new Mutation("row1");
-    m.put("colf", "colq", 3l, "value");
-    m.put("colf", "colq", 4l, "value");
-    bw.addMutation(m);
-    bw.flush();
-
-    final Scanner scanner = conn.createScanner("table", new Authorizations());
-
-    final ClientSideIteratorScanner csis = new ClientSideIteratorScanner(scanner);
-    final IteratorSetting si = new IteratorSetting(10, "localvers", VersioningIterator.class);
-    si.addOption("maxVersions", "2");
-    csis.addScanIterator(si);
-
-    checkResults(csis, resultSet1, PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME);
-    checkResults(scanner, resultSet2, PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME);
-
-    csis.fetchColumnFamily(new Text("colf"));
-    checkResults(csis, resultSet1, PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME);
-    csis.clearColumns();
-    csis.fetchColumnFamily(new Text("none"));
-    assertFalse(csis.iterator().hasNext());
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
index 9bb85f4..bcf8a24 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
@@ -17,199 +17,16 @@
 package org.apache.accumulo.core.client.mapred;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
-import java.io.File;
-import java.io.FileFilter;
 import java.io.IOException;
 
-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.mapreduce.lib.impl.ConfiguratorBase;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator;
-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.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.lib.IdentityMapper;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.BeforeClass;
-import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class AccumuloFileOutputFormatTest {
-  private static final Logger log = LoggerFactory.getLogger(AccumuloFileOutputFormatTest.class);
-  private static final int JOB_VISIBILITY_CACHE_SIZE = 3000;
-  private static final String PREFIX = AccumuloFileOutputFormatTest.class.getSimpleName();
-  private static final String INSTANCE_NAME = PREFIX + "_mapred_instance";
-  private static final String BAD_TABLE = PREFIX + "_mapred_bad_table";
-  private static final String TEST_TABLE = PREFIX + "_mapred_test_table";
-  private static final String EMPTY_TABLE = PREFIX + "_mapred_empty_table";
-
-  private static AssertionError e1 = null;
-  private static AssertionError e2 = null;
-
-  @Rule
-  public TemporaryFolder folder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
-
-  @BeforeClass
-  public static void setup() throws Exception {
-    Instance mockInstance = new org.apache.accumulo.core.client.mock.MockInstance(INSTANCE_NAME);
-    Connector c = mockInstance.getConnector("root", new PasswordToken(""));
-    c.tableOperations().create(EMPTY_TABLE);
-    c.tableOperations().create(TEST_TABLE);
-    c.tableOperations().create(BAD_TABLE);
-    BatchWriter bw = c.createBatchWriter(TEST_TABLE, new BatchWriterConfig());
-    Mutation m = new Mutation("Key");
-    m.put("", "", "");
-    bw.addMutation(m);
-    bw.close();
-    bw = c.createBatchWriter(BAD_TABLE, new BatchWriterConfig());
-    m = new Mutation("r1");
-    m.put("cf1", "cq1", "A&B");
-    m.put("cf1", "cq1", "A&B");
-    m.put("cf1", "cq2", "A&");
-    bw.addMutation(m);
-    bw.close();
-  }
-
-  @Test
-  public void testEmptyWrite() throws Exception {
-    handleWriteTests(false);
-  }
-
-  @Test
-  public void testRealWrite() throws Exception {
-    handleWriteTests(true);
-  }
-
-  private static class MRTester extends Configured implements Tool {
-    private static class BadKeyMapper implements Mapper<Key,Value,Key,Value> {
-
-      int index = 0;
-
-      @Override
-      public void map(Key key, Value value, OutputCollector<Key,Value> output, Reporter reporter) throws IOException {
-        try {
-          try {
-            output.collect(key, value);
-            if (index == 2)
-              fail();
-          } catch (Exception e) {
-            log.error(e.toString(), e);
-            assertEquals(2, index);
-          }
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        index++;
-      }
-
-      @Override
-      public void configure(JobConf job) {}
-
-      @Override
-      public void close() throws IOException {
-        try {
-          assertEquals(2, index);
-        } catch (AssertionError e) {
-          e2 = e;
-        }
-      }
-
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 4) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table> <outputfile>");
-      }
-
-      String user = args[0];
-      String pass = args[1];
-      String table = args[2];
-
-      JobConf job = new JobConf(getConf());
-      job.setJarByClass(this.getClass());
-      ConfiguratorBase.setVisibilityCacheSize(job, JOB_VISIBILITY_CACHE_SIZE);
-
-      job.setInputFormat(AccumuloInputFormat.class);
-
-      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
-      AccumuloInputFormat.setInputTableName(job, table);
-      AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
-      AccumuloFileOutputFormat.setOutputPath(job, new Path(args[3]));
-
-      job.setMapperClass(BAD_TABLE.equals(table) ? BadKeyMapper.class : IdentityMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormat(AccumuloFileOutputFormat.class);
-
-      job.setNumReduceTasks(0);
-
-      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
-
-  public void handleWriteTests(boolean content) throws Exception {
-    File f = folder.newFile("handleWriteTests");
-    if (f.delete()) {
-      log.debug("Deleted {}", f);
-    }
-    MRTester.main(new String[] {"root", "", content ? TEST_TABLE : EMPTY_TABLE, f.getAbsolutePath()});
-
-    assertTrue(f.exists());
-    File[] files = f.listFiles(new FileFilter() {
-      @Override
-      public boolean accept(File file) {
-        return file.getName().startsWith("part-m-");
-      }
-    });
-    assertNotNull(files);
-    if (content) {
-      assertEquals(1, files.length);
-      assertTrue(files[0].exists());
-    } else {
-      assertEquals(0, files.length);
-    }
-  }
-
-  @Test
-  public void writeBadVisibility() throws Exception {
-    File f = folder.newFile("writeBadVisibility");
-    if (f.delete()) {
-      log.debug("Deleted {}", f);
-    }
-    MRTester.main(new String[] {"root", "", BAD_TABLE, f.getAbsolutePath()});
-    assertNull(e1);
-    assertNull(e2);
-  }
 
   @Test
   public void validateConfiguration() throws IOException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
index ad9f419..cb28958 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormatTest.java
@@ -17,73 +17,30 @@
 package org.apache.accumulo.core.client.mapred;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
-import java.io.File;
 import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
 
-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.security.tokens.PasswordToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
-import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.Base64;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.lib.NullOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.log4j.Level;
-import org.junit.Assert;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
 
 public class AccumuloInputFormatTest {
 
-  private static final String PREFIX = AccumuloInputFormatTest.class.getSimpleName();
-  private static final String TEST_TABLE_1 = PREFIX + "_mapred_table_1";
-
   private JobConf job;
 
-  @BeforeClass
-  public static void setupClass() {
-    System.setProperty("hadoop.tmp.dir", System.getProperty("user.dir") + "/target/hadoop-tmp");
-  }
-
   @Rule
   public TestName test = new TestName();
 
-  private Instance inst;
-
-  @Before
-  public void setupInstance() throws Exception {
-    inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-  }
-
   @Before
   public void createJob() {
     job = new JobConf();
@@ -213,140 +170,4 @@ public class AccumuloInputFormatTest {
     assertTrue(regex.equals(AccumuloInputFormat.getIterators(job).get(0).getName()));
   }
 
-  private static AssertionError e1 = null;
-  private static AssertionError e2 = null;
-
-  private static class MRTester extends Configured implements Tool {
-    private static class TestMapper implements Mapper<Key,Value,Key,Value> {
-      Key key = null;
-      int count = 0;
-
-      @Override
-      public void map(Key k, Value v, OutputCollector<Key,Value> output, Reporter reporter) throws IOException {
-        try {
-          if (key != null)
-            assertEquals(key.getRow().toString(), new String(v.get()));
-          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
-          assertEquals(new String(v.get()), String.format("%09x", count));
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        key = new Key(k);
-        count++;
-      }
-
-      @Override
-      public void configure(JobConf job) {}
-
-      @Override
-      public void close() throws IOException {
-        try {
-          assertEquals(100, count);
-        } catch (AssertionError e) {
-          e2 = e;
-        }
-      }
-
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 4) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table> <instanceName>");
-      }
-
-      String user = args[0];
-      String pass = args[1];
-      String table = args[2];
-      String instanceName = args[3];
-
-      JobConf job = new JobConf(getConf());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormat(AccumuloInputFormat.class);
-
-      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
-      AccumuloInputFormat.setInputTableName(job, table);
-      AccumuloInputFormat.setMockInstance(job, instanceName);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormat(NullOutputFormat.class);
-
-      job.setNumReduceTasks(0);
-
-      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String... args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
-
-  @Test
-  public void testMap() throws Exception {
-    Connector c = inst.getConnector("root", new PasswordToken(""));
-    c.tableOperations().create(TEST_TABLE_1);
-    BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
-    for (int i = 0; i < 100; i++) {
-      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
-      bw.addMutation(m);
-    }
-    bw.close();
-
-    MRTester.main("root", "", TEST_TABLE_1, inst.getInstanceName());
-    assertNull(e1);
-    assertNull(e2);
-  }
-
-  @Test
-  public void testCorrectRangeInputSplits() throws Exception {
-    JobConf job = new JobConf();
-
-    String username = "user", table = "table";
-    PasswordToken password = new PasswordToken("password");
-    Authorizations auths = new Authorizations("foo");
-    Collection<Pair<Text,Text>> fetchColumns = Collections.singleton(new Pair<Text,Text>(new Text("foo"), new Text("bar")));
-    boolean isolated = true, localIters = true;
-    Level level = Level.WARN;
-
-    Connector connector = inst.getConnector(username, password);
-    connector.tableOperations().create(table);
-
-    AccumuloInputFormat.setConnectorInfo(job, username, password);
-    AccumuloInputFormat.setInputTableName(job, table);
-    AccumuloInputFormat.setScanAuthorizations(job, auths);
-    AccumuloInputFormat.setMockInstance(job, inst.getInstanceName());
-    AccumuloInputFormat.setScanIsolation(job, isolated);
-    AccumuloInputFormat.setLocalIterators(job, localIters);
-    AccumuloInputFormat.fetchColumns(job, fetchColumns);
-    AccumuloInputFormat.setLogLevel(job, level);
-
-    AccumuloInputFormat aif = new AccumuloInputFormat();
-
-    InputSplit[] splits = aif.getSplits(job, 1);
-
-    Assert.assertEquals(1, splits.length);
-
-    InputSplit split = splits[0];
-
-    Assert.assertEquals(RangeInputSplit.class, split.getClass());
-
-    RangeInputSplit risplit = (RangeInputSplit) split;
-
-    Assert.assertEquals(username, risplit.getPrincipal());
-    Assert.assertEquals(table, risplit.getTableName());
-    Assert.assertEquals(password, risplit.getToken());
-    Assert.assertEquals(auths, risplit.getAuths());
-    Assert.assertEquals(inst.getInstanceName(), risplit.getInstanceName());
-    Assert.assertEquals(isolated, risplit.isIsolatedScan());
-    Assert.assertEquals(localIters, risplit.usesLocalIterators());
-    Assert.assertEquals(fetchColumns, risplit.getFetchedColumns());
-    Assert.assertEquals(level, risplit.getLogLevel());
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
index f9cca65..ef19901 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormatTest.java
@@ -17,159 +17,34 @@
 package org.apache.accumulo.core.client.mapred;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
-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.mapreduce.InputTableConfig;
-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.util.Pair;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.lib.NullOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TestName;
 
 public class AccumuloMultiTableInputFormatTest {
 
-  private static final String PREFIX = AccumuloMultiTableInputFormatTest.class.getSimpleName();
-  private static final String INSTANCE_NAME = PREFIX + "_mapred_instance";
-  private static final String TEST_TABLE_1 = PREFIX + "_mapred_table_1";
-  private static final String TEST_TABLE_2 = PREFIX + "_mapred_table_2";
-
-  private static AssertionError e1 = null;
-  private static AssertionError e2 = null;
-
-  private static class MRTester extends Configured implements Tool {
-    private static class TestMapper implements Mapper<Key,Value,Key,Value> {
-      Key key = null;
-      int count = 0;
-
-      @Override
-      public void map(Key k, Value v, OutputCollector<Key,Value> output, Reporter reporter) throws IOException {
-        try {
-          String tableName = ((RangeInputSplit) reporter.getInputSplit()).getTableName();
-          if (key != null)
-            assertEquals(key.getRow().toString(), new String(v.get()));
-          assertEquals(new Text(String.format("%s_%09x", tableName, count + 1)), k.getRow());
-          assertEquals(String.format("%s_%09x", tableName, count), new String(v.get()));
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        key = new Key(k);
-        count++;
-      }
-
-      @Override
-      public void configure(JobConf job) {}
-
-      @Override
-      public void close() throws IOException {
-        try {
-          assertEquals(100, count);
-        } catch (AssertionError e) {
-          e2 = e;
-        }
-      }
-
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 4) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table1> <table2>");
-      }
-
-      String user = args[0];
-      String pass = args[1];
-      String table1 = args[2];
-      String table2 = args[3];
-
-      JobConf job = new JobConf(getConf());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormat(AccumuloInputFormat.class);
-
-      AccumuloMultiTableInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
-      AccumuloMultiTableInputFormat.setMockInstance(job, INSTANCE_NAME);
-
-      InputTableConfig tableConfig1 = new InputTableConfig();
-      InputTableConfig tableConfig2 = new InputTableConfig();
-
-      Map<String,InputTableConfig> configMap = new HashMap<String,InputTableConfig>();
-      configMap.put(table1, tableConfig1);
-      configMap.put(table2, tableConfig2);
-
-      AccumuloMultiTableInputFormat.setInputTableConfigs(job, configMap);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormat(NullOutputFormat.class);
-
-      job.setNumReduceTasks(0);
-
-      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
-
-  @Test
-  public void testMap() throws Exception {
-    Instance mockInstance = new org.apache.accumulo.core.client.mock.MockInstance(INSTANCE_NAME);
-    Connector c = mockInstance.getConnector("root", new PasswordToken(""));
-    c.tableOperations().create(TEST_TABLE_1);
-    c.tableOperations().create(TEST_TABLE_2);
-    BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
-    BatchWriter bw2 = c.createBatchWriter(TEST_TABLE_2, new BatchWriterConfig());
-    for (int i = 0; i < 100; i++) {
-      Mutation t1m = new Mutation(new Text(String.format("%s_%09x", TEST_TABLE_1, i + 1)));
-      t1m.put(new Text(), new Text(), new Value(String.format("%s_%09x", TEST_TABLE_1, i).getBytes()));
-      bw.addMutation(t1m);
-      Mutation t2m = new Mutation(new Text(String.format("%s_%09x", TEST_TABLE_2, i + 1)));
-      t2m.put(new Text(), new Text(), new Value(String.format("%s_%09x", TEST_TABLE_2, i).getBytes()));
-      bw2.addMutation(t2m);
-    }
-    bw.close();
-    bw2.close();
-
-    MRTester.main(new String[] {"root", "", TEST_TABLE_1, TEST_TABLE_2});
-    assertNull(e1);
-    assertNull(e2);
-  }
+  @Rule
+  public TestName testName = new TestName();
 
   /**
    * Verify {@link org.apache.accumulo.core.client.mapreduce.InputTableConfig} objects get correctly serialized in the JobContext.
    */
   @Test
   public void testTableQueryConfigSerialization() throws IOException {
-
+    String table1Name = testName.getMethodName() + "1";
+    String table2Name = testName.getMethodName() + "2";
     JobConf job = new JobConf();
 
     InputTableConfig table1 = new InputTableConfig().setRanges(Collections.singletonList(new Range("a", "b")))
@@ -181,11 +56,11 @@ public class AccumuloMultiTableInputFormatTest {
         .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
 
     Map<String,InputTableConfig> configMap = new HashMap<String,InputTableConfig>();
-    configMap.put(TEST_TABLE_1, table1);
-    configMap.put(TEST_TABLE_2, table2);
+    configMap.put(table1Name, table1);
+    configMap.put(table2Name, table2);
     AccumuloMultiTableInputFormat.setInputTableConfigs(job, configMap);
 
-    assertEquals(table1, AccumuloMultiTableInputFormat.getInputTableConfig(job, TEST_TABLE_1));
-    assertEquals(table2, AccumuloMultiTableInputFormat.getInputTableConfig(job, TEST_TABLE_2));
+    assertEquals(table1, AccumuloMultiTableInputFormat.getInputTableConfig(job, table1Name));
+    assertEquals(table2, AccumuloMultiTableInputFormat.getInputTableConfig(job, table2Name));
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java
index d1f4c4c..ab73cfe 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormatTest.java
@@ -1,12 +1,12 @@
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
+ * 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
+ * the License. You may obtain a copy of the License at
  *
- *     http://www.apache.org/licenses/LICENSE-2.0
+ * 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,
@@ -17,127 +17,17 @@
 package org.apache.accumulo.core.client.mapred;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
 
-import java.io.File;
 import java.io.IOException;
-import java.util.Iterator;
-import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.Scanner;
-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.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
 
-/**
- *
- */
 public class AccumuloOutputFormatTest {
-  private static AssertionError e1 = null;
-  private static final String PREFIX = AccumuloOutputFormatTest.class.getSimpleName();
-  private static final String INSTANCE_NAME = PREFIX + "_mapred_instance";
-  private static final String TEST_TABLE_1 = PREFIX + "_mapred_table_1";
-  private static final String TEST_TABLE_2 = PREFIX + "_mapred_table_2";
-
-  private static class MRTester extends Configured implements Tool {
-    private static class TestMapper implements Mapper<Key,Value,Text,Mutation> {
-      Key key = null;
-      int count = 0;
-      OutputCollector<Text,Mutation> finalOutput;
-
-      @Override
-      public void map(Key k, Value v, OutputCollector<Text,Mutation> output, Reporter reporter) throws IOException {
-        finalOutput = output;
-        try {
-          if (key != null)
-            assertEquals(key.getRow().toString(), new String(v.get()));
-          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
-          assertEquals(new String(v.get()), String.format("%09x", count));
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        key = new Key(k);
-        count++;
-      }
-
-      @Override
-      public void configure(JobConf job) {}
-
-      @Override
-      public void close() throws IOException {
-        Mutation m = new Mutation("total");
-        m.put("", "", Integer.toString(count));
-        finalOutput.collect(new Text(), m);
-      }
-
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 4) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <inputtable> <outputtable>");
-      }
-
-      String user = args[0];
-      String pass = args[1];
-      String table1 = args[2];
-      String table2 = args[3];
-
-      JobConf job = new JobConf(getConf());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormat(AccumuloInputFormat.class);
-
-      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
-      AccumuloInputFormat.setInputTableName(job, table1);
-      AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormat(AccumuloOutputFormat.class);
-      job.setOutputKeyClass(Text.class);
-      job.setOutputValueClass(Mutation.class);
-
-      AccumuloOutputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
-      AccumuloOutputFormat.setCreateTables(job, false);
-      AccumuloOutputFormat.setDefaultTableName(job, table2);
-      AccumuloOutputFormat.setMockInstance(job, INSTANCE_NAME);
-
-      job.setNumReduceTasks(0);
-
-      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
 
   @Test
   public void testBWSettings() throws IOException {
@@ -179,28 +69,4 @@ public class AccumuloOutputFormatTest {
     myAOF.checkOutputSpecs(null, job);
   }
 
-  @Test
-  public void testMR() throws Exception {
-    Instance mockInstance = new org.apache.accumulo.core.client.mock.MockInstance(INSTANCE_NAME);
-    Connector c = mockInstance.getConnector("root", new PasswordToken(""));
-    c.tableOperations().create(TEST_TABLE_1);
-    c.tableOperations().create(TEST_TABLE_2);
-    BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
-    for (int i = 0; i < 100; i++) {
-      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
-      bw.addMutation(m);
-    }
-    bw.close();
-
-    MRTester.main(new String[] {"root", "", TEST_TABLE_1, TEST_TABLE_2});
-    assertNull(e1);
-
-    Scanner scanner = c.createScanner(TEST_TABLE_2, new Authorizations());
-    Iterator<Entry<Key,Value>> iter = scanner.iterator();
-    assertTrue(iter.hasNext());
-    Entry<Key,Value> entry = iter.next();
-    assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
-    assertFalse(iter.hasNext());
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java
deleted file mode 100644
index 3d59149..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormatTest.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map.Entry;
-
-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.MutationsRejectedException;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyValue;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.util.PeekingIterator;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.lib.NullOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-public class AccumuloRowInputFormatTest {
-  private static final String PREFIX = AccumuloRowInputFormatTest.class.getSimpleName();
-  private static final String TEST_TABLE_1 = PREFIX + "_mapred_table_1";
-
-  private static final String ROW1 = "row1";
-  private static final String ROW2 = "row2";
-  private static final String ROW3 = "row3";
-  private static final String COLF1 = "colf1";
-  private static final List<Entry<Key,Value>> row1;
-  private static final List<Entry<Key,Value>> row2;
-  private static final List<Entry<Key,Value>> row3;
-  private static AssertionError e1 = null;
-  private static AssertionError e2 = null;
-
-  static {
-    row1 = new ArrayList<Entry<Key,Value>>();
-    row1.add(new KeyValue(new Key(ROW1, COLF1, "colq1"), "v1".getBytes()));
-    row1.add(new KeyValue(new Key(ROW1, COLF1, "colq2"), "v2".getBytes()));
-    row1.add(new KeyValue(new Key(ROW1, "colf2", "colq3"), "v3".getBytes()));
-    row2 = new ArrayList<Entry<Key,Value>>();
-    row2.add(new KeyValue(new Key(ROW2, COLF1, "colq4"), "v4".getBytes()));
-    row3 = new ArrayList<Entry<Key,Value>>();
-    row3.add(new KeyValue(new Key(ROW3, COLF1, "colq5"), "v5".getBytes()));
-  }
-
-  public static void checkLists(final List<Entry<Key,Value>> first, final List<Entry<Key,Value>> second) {
-    assertEquals("Sizes should be the same.", first.size(), second.size());
-    for (int i = 0; i < first.size(); i++) {
-      assertEquals("Keys should be equal.", first.get(i).getKey(), second.get(i).getKey());
-      assertEquals("Values should be equal.", first.get(i).getValue(), second.get(i).getValue());
-    }
-  }
-
-  public static void checkLists(final List<Entry<Key,Value>> first, final Iterator<Entry<Key,Value>> second) {
-    int entryIndex = 0;
-    while (second.hasNext()) {
-      final Entry<Key,Value> entry = second.next();
-      assertEquals("Keys should be equal", first.get(entryIndex).getKey(), entry.getKey());
-      assertEquals("Values should be equal", first.get(entryIndex).getValue(), entry.getValue());
-      entryIndex++;
-    }
-  }
-
-  public static void insertList(final BatchWriter writer, final List<Entry<Key,Value>> list) throws MutationsRejectedException {
-    for (Entry<Key,Value> e : list) {
-      final Key key = e.getKey();
-      final Mutation mutation = new Mutation(key.getRow());
-      ColumnVisibility colVisibility = new ColumnVisibility(key.getColumnVisibility());
-      mutation.put(key.getColumnFamily(), key.getColumnQualifier(), colVisibility, key.getTimestamp(), e.getValue());
-      writer.addMutation(mutation);
-    }
-  }
-
-  private static class MRTester extends Configured implements Tool {
-    public static class TestMapper implements Mapper<Text,PeekingIterator<Entry<Key,Value>>,Key,Value> {
-      int count = 0;
-
-      @Override
-      public void map(Text k, PeekingIterator<Entry<Key,Value>> v, OutputCollector<Key,Value> output, Reporter reporter) throws IOException {
-        try {
-          switch (count) {
-            case 0:
-              assertEquals("Current key should be " + ROW1, new Text(ROW1), k);
-              checkLists(row1, v);
-              break;
-            case 1:
-              assertEquals("Current key should be " + ROW2, new Text(ROW2), k);
-              checkLists(row2, v);
-              break;
-            case 2:
-              assertEquals("Current key should be " + ROW3, new Text(ROW3), k);
-              checkLists(row3, v);
-              break;
-            default:
-              assertTrue(false);
-          }
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        count++;
-      }
-
-      @Override
-      public void configure(JobConf job) {}
-
-      @Override
-      public void close() throws IOException {
-        try {
-          assertEquals(3, count);
-        } catch (AssertionError e) {
-          e2 = e;
-        }
-      }
-
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 4) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table> <instanceName>");
-      }
-
-      String user = args[0];
-      String pass = args[1];
-      String table = args[2];
-      String instanceName = args[3];
-
-      JobConf job = new JobConf(getConf());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormat(AccumuloRowInputFormat.class);
-
-      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
-      AccumuloInputFormat.setInputTableName(job, table);
-      AccumuloRowInputFormat.setMockInstance(job, instanceName);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormat(NullOutputFormat.class);
-
-      job.setNumReduceTasks(0);
-
-      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
-
-  @Rule
-  public TestName test = new TestName();
-
-  @Test
-  public void test() throws Exception {
-    final Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    final Connector conn = inst.getConnector("root", new PasswordToken(""));
-    conn.tableOperations().create(TEST_TABLE_1);
-    BatchWriter writer = null;
-    try {
-      writer = conn.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
-      insertList(writer, row1);
-      insertList(writer, row2);
-      insertList(writer, row3);
-    } finally {
-      if (writer != null) {
-        writer.close();
-      }
-    }
-    MRTester.main(new String[] {"root", "", TEST_TABLE_1, inst.getInstanceName()});
-    assertNull(e1);
-    assertNull(e2);
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java
deleted file mode 100644
index 7b4ef75..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/TokenFileTest.java
+++ /dev/null
@@ -1,179 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.client.mapred;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.util.Iterator;
-import java.util.Map.Entry;
-
-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.Scanner;
-import org.apache.accumulo.core.client.impl.Credentials;
-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.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.CachedConfiguration;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobClient;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Mapper;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.rules.TestName;
-
-public class TokenFileTest {
-  private static AssertionError e1 = null;
-  private static final String PREFIX = TokenFileTest.class.getSimpleName();
-  private static final String TEST_TABLE_1 = PREFIX + "_mapred_table_1";
-  private static final String TEST_TABLE_2 = PREFIX + "_mapred_table_2";
-
-  private static class MRTokenFileTester extends Configured implements Tool {
-    private static class TestMapper implements Mapper<Key,Value,Text,Mutation> {
-      Key key = null;
-      int count = 0;
-      OutputCollector<Text,Mutation> finalOutput;
-
-      @Override
-      public void map(Key k, Value v, OutputCollector<Text,Mutation> output, Reporter reporter) throws IOException {
-        finalOutput = output;
-        try {
-          if (key != null)
-            assertEquals(key.getRow().toString(), new String(v.get()));
-          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
-          assertEquals(new String(v.get()), String.format("%09x", count));
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        key = new Key(k);
-        count++;
-      }
-
-      @Override
-      public void configure(JobConf job) {}
-
-      @Override
-      public void close() throws IOException {
-        Mutation m = new Mutation("total");
-        m.put("", "", Integer.toString(count));
-        finalOutput.collect(new Text(), m);
-      }
-
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 5) {
-        throw new IllegalArgumentException("Usage : " + MRTokenFileTester.class.getName() + " <user> <token file> <inputtable> <outputtable> <instanceName>");
-      }
-
-      String user = args[0];
-      String tokenFile = args[1];
-      String table1 = args[2];
-      String table2 = args[3];
-      String instanceName = args[4];
-
-      JobConf job = new JobConf(getConf());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormat(AccumuloInputFormat.class);
-
-      AccumuloInputFormat.setConnectorInfo(job, user, tokenFile);
-      AccumuloInputFormat.setInputTableName(job, table1);
-      AccumuloInputFormat.setMockInstance(job, instanceName);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormat(AccumuloOutputFormat.class);
-      job.setOutputKeyClass(Text.class);
-      job.setOutputValueClass(Mutation.class);
-
-      AccumuloOutputFormat.setConnectorInfo(job, user, tokenFile);
-      AccumuloOutputFormat.setCreateTables(job, false);
-      AccumuloOutputFormat.setDefaultTableName(job, table2);
-      AccumuloOutputFormat.setMockInstance(job, instanceName);
-
-      job.setNumReduceTasks(0);
-
-      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = CachedConfiguration.getInstance();
-      conf.set("hadoop.tmp.dir", new File(args[1]).getParent());
-      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTokenFileTester(), args));
-    }
-  }
-
-  @Rule
-  public TemporaryFolder folder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
-
-  @Rule
-  public TestName test = new TestName();
-
-  @Test
-  public void testMR() throws Exception {
-    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    Connector c = inst.getConnector("root", new PasswordToken(""));
-    c.tableOperations().create(TEST_TABLE_1);
-    c.tableOperations().create(TEST_TABLE_2);
-    BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
-    for (int i = 0; i < 100; i++) {
-      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
-      bw.addMutation(m);
-    }
-    bw.close();
-
-    File tf = folder.newFile("root_test.pw");
-    PrintStream out = new PrintStream(tf);
-    String outString = new Credentials("root", new PasswordToken("")).serialize();
-    out.println(outString);
-    out.close();
-
-    MRTokenFileTester.main(new String[] {"root", tf.getAbsolutePath(), TEST_TABLE_1, TEST_TABLE_2, inst.getInstanceName()});
-    assertNull(e1);
-
-    Scanner scanner = c.createScanner(TEST_TABLE_2, new Authorizations());
-    Iterator<Entry<Key,Value>> iter = scanner.iterator();
-    assertTrue(iter.hasNext());
-    Entry<Key,Value> entry = iter.next();
-    assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
-    assertFalse(iter.hasNext());
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
index 43b21ce..3923566 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloFileOutputFormatTest.java
@@ -17,187 +17,16 @@
 package org.apache.accumulo.core.client.mapreduce;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
 
-import java.io.File;
-import java.io.FileFilter;
 import java.io.IOException;
 
-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.mapreduce.lib.impl.FileOutputConfigurator;
-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.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.BeforeClass;
-import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 public class AccumuloFileOutputFormatTest {
-  private static final Logger log = LoggerFactory.getLogger(AccumuloFileOutputFormatTest.class);
-  private static final String PREFIX = AccumuloFileOutputFormatTest.class.getSimpleName();
-  private static final String INSTANCE_NAME = PREFIX + "_mapreduce_instance";
-  private static final String BAD_TABLE = PREFIX + "_mapreduce_bad_table";
-  private static final String TEST_TABLE = PREFIX + "_mapreduce_test_table";
-  private static final String EMPTY_TABLE = PREFIX + "_mapreduce_empty_table";
-
-  private static AssertionError e1 = null;
-  private static AssertionError e2 = null;
-
-  @Rule
-  public TemporaryFolder folder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
-
-  @BeforeClass
-  public static void setup() throws Exception {
-    Instance mockInstance = new org.apache.accumulo.core.client.mock.MockInstance(INSTANCE_NAME);
-    Connector c = mockInstance.getConnector("root", new PasswordToken(""));
-    c.tableOperations().create(EMPTY_TABLE);
-    c.tableOperations().create(TEST_TABLE);
-    c.tableOperations().create(BAD_TABLE);
-    BatchWriter bw = c.createBatchWriter(TEST_TABLE, new BatchWriterConfig());
-    Mutation m = new Mutation("Key");
-    m.put("", "", "");
-    bw.addMutation(m);
-    bw.close();
-    bw = c.createBatchWriter(BAD_TABLE, new BatchWriterConfig());
-    m = new Mutation("r1");
-    m.put("cf1", "cq1", "A&B");
-    m.put("cf1", "cq1", "A&B");
-    m.put("cf1", "cq2", "A&");
-    bw.addMutation(m);
-    bw.close();
-  }
-
-  @Test
-  public void testEmptyWrite() throws Exception {
-    handleWriteTests(false);
-  }
-
-  @Test
-  public void testRealWrite() throws Exception {
-    handleWriteTests(true);
-  }
-
-  private static class MRTester extends Configured implements Tool {
-    private static class BadKeyMapper extends Mapper<Key,Value,Key,Value> {
-      int index = 0;
-
-      @Override
-      protected void map(Key key, Value value, Context context) throws IOException, InterruptedException {
-        try {
-          try {
-            context.write(key, value);
-            if (index == 2)
-              assertTrue(false);
-          } catch (Exception e) {
-            assertEquals(2, index);
-          }
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        index++;
-      }
-
-      @Override
-      protected void cleanup(Context context) throws IOException, InterruptedException {
-        try {
-          assertEquals(2, index);
-        } catch (AssertionError e) {
-          e2 = e;
-        }
-      }
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 4) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table> <outputfile>");
-      }
-
-      String user = args[0];
-      String pass = args[1];
-      String table = args[2];
-
-      Job job = Job.getInstance(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormatClass(AccumuloInputFormat.class);
-
-      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
-      AccumuloInputFormat.setInputTableName(job, table);
-      AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
-      AccumuloFileOutputFormat.setOutputPath(job, new Path(args[3]));
-
-      job.setMapperClass(BAD_TABLE.equals(table) ? BadKeyMapper.class : Mapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormatClass(AccumuloFileOutputFormat.class);
-
-      job.setNumReduceTasks(0);
-
-      job.waitForCompletion(true);
-
-      return job.isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
-
-  public void handleWriteTests(boolean content) throws Exception {
-    File f = folder.newFile("handleWriteTests");
-    if (f.delete()) {
-      log.debug("Deleted {}", f);
-    }
-    MRTester.main(new String[] {"root", "", content ? TEST_TABLE : EMPTY_TABLE, f.getAbsolutePath()});
-
-    assertTrue(f.exists());
-    File[] files = f.listFiles(new FileFilter() {
-      @Override
-      public boolean accept(File file) {
-        return file.getName().startsWith("part-m-");
-      }
-    });
-    assertNotNull(files);
-    if (content) {
-      assertEquals(1, files.length);
-      assertTrue(files[0].exists());
-    } else {
-      assertEquals(0, files.length);
-    }
-  }
-
-  @Test
-  public void writeBadVisibility() throws Exception {
-    File f = folder.newFile("writeBadVisibility");
-    if (f.delete()) {
-      log.debug("Deleted {}", f);
-    }
-    MRTester.main(new String[] {"root", "", BAD_TABLE, f.getAbsolutePath()});
-    assertNull(e1);
-    assertNull(e2);
-  }
 
   @Test
   public void validateConfiguration() throws IOException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
index ff5e5f6..56c8195 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormatTest.java
@@ -17,62 +17,27 @@
 package org.apache.accumulo.core.client.mapreduce;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
-import java.io.File;
 import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
-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.security.tokens.PasswordToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
-import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.Base64;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.log4j.Level;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TestName;
 
 public class AccumuloInputFormatTest {
 
-  @Rule
-  public TestName test = new TestName();
-
-  private Instance inst;
-
-  @Before
-  public void setupInstance() throws Exception {
-    inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-  }
-
   /**
    * Check that the iterator configuration is getting stored in the Job conf correctly.
    */
@@ -207,217 +172,6 @@ public class AccumuloInputFormatTest {
     assertTrue(regex.equals(AccumuloInputFormat.getIterators(job).get(0).getName()));
   }
 
-  private static AssertionError e1 = null;
-  private static AssertionError e2 = null;
-
-  private static class MRTester extends Configured implements Tool {
-    private static class TestMapper extends Mapper<Key,Value,Key,Value> {
-      Key key = null;
-      int count = 0;
-
-      @Override
-      protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
-        try {
-          if (key != null)
-            assertEquals(key.getRow().toString(), new String(v.get()));
-          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
-          assertEquals(new String(v.get()), String.format("%09x", count));
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        key = new Key(k);
-        count++;
-      }
-
-      @Override
-      protected void cleanup(Context context) throws IOException, InterruptedException {
-        try {
-          assertEquals(100, count);
-        } catch (AssertionError e) {
-          e2 = e;
-        }
-      }
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 5 && args.length != 6) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table> <instanceName> <inputFormatClass> [<batchScan>]");
-      }
-
-      String user = args[0];
-      String pass = args[1];
-      String table = args[2];
-
-      String instanceName = args[3];
-      String inputFormatClassName = args[4];
-      Boolean batchScan = false;
-      if (args.length == 6)
-        batchScan = Boolean.parseBoolean(args[5]);
-
-      @SuppressWarnings("unchecked")
-      Class<? extends InputFormat<?,?>> inputFormatClass = (Class<? extends InputFormat<?,?>>) Class.forName(inputFormatClassName);
-
-      Job job = Job.getInstance(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormatClass(inputFormatClass);
-
-      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
-      AccumuloInputFormat.setInputTableName(job, table);
-      AccumuloInputFormat.setMockInstance(job, instanceName);
-      AccumuloInputFormat.setBatchScan(job, batchScan);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormatClass(NullOutputFormat.class);
-
-      job.setNumReduceTasks(0);
-
-      job.waitForCompletion(true);
-
-      return job.isSuccessful() ? 0 : 1;
-    }
-
-    public static int main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      return ToolRunner.run(conf, new MRTester(), args);
-    }
-  }
-
-  @Test
-  public void testMap() throws Exception {
-    final String TEST_TABLE_1 = test.getMethodName() + "_mapreduce_table_1";
-
-    Connector c = inst.getConnector("root", new PasswordToken(""));
-    c.tableOperations().create(TEST_TABLE_1);
-    BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
-    for (int i = 0; i < 100; i++) {
-      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
-      bw.addMutation(m);
-    }
-    bw.close();
-
-    Assert.assertEquals(0, MRTester.main(new String[] {"root", "", TEST_TABLE_1, inst.getInstanceName(), AccumuloInputFormat.class.getCanonicalName()}));
-    assertNull(e1);
-    assertNull(e2);
-  }
-
-  @Test
-  public void testMapWithBatchScanner() throws Exception {
-    final String TEST_TABLE_2 = test.getMethodName() + "_mapreduce_table_2";
-
-    Connector c = inst.getConnector("root", new PasswordToken(""));
-    c.tableOperations().create(TEST_TABLE_2);
-    BatchWriter bw = c.createBatchWriter(TEST_TABLE_2, new BatchWriterConfig());
-    for (int i = 0; i < 100; i++) {
-      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
-      bw.addMutation(m);
-    }
-    bw.close();
-
-    Assert
-        .assertEquals(0, MRTester.main(new String[] {"root", "", TEST_TABLE_2, inst.getInstanceName(), AccumuloInputFormat.class.getCanonicalName(), "True"}));
-    assertNull(e1);
-    assertNull(e2);
-  }
-
-  @Test
-  public void testCorrectRangeInputSplits() throws Exception {
-    Job job = Job.getInstance(new Configuration(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
-
-    String username = "user", table = "table";
-    PasswordToken password = new PasswordToken("password");
-    Authorizations auths = new Authorizations("foo");
-    Collection<Pair<Text,Text>> fetchColumns = Collections.singleton(new Pair<Text,Text>(new Text("foo"), new Text("bar")));
-    boolean isolated = true, localIters = true;
-    Level level = Level.WARN;
-
-    Connector connector = inst.getConnector(username, password);
-    connector.tableOperations().create(table);
-
-    AccumuloInputFormat.setConnectorInfo(job, username, password);
-    AccumuloInputFormat.setInputTableName(job, table);
-    AccumuloInputFormat.setScanAuthorizations(job, auths);
-    AccumuloInputFormat.setMockInstance(job, inst.getInstanceName());
-    AccumuloInputFormat.setScanIsolation(job, isolated);
-    AccumuloInputFormat.setLocalIterators(job, localIters);
-    AccumuloInputFormat.fetchColumns(job, fetchColumns);
-    AccumuloInputFormat.setLogLevel(job, level);
-
-    AccumuloInputFormat aif = new AccumuloInputFormat();
-
-    List<InputSplit> splits = aif.getSplits(job);
-
-    Assert.assertEquals(1, splits.size());
-
-    InputSplit split = splits.get(0);
-
-    Assert.assertEquals(RangeInputSplit.class, split.getClass());
-
-    RangeInputSplit risplit = (RangeInputSplit) split;
-
-    Assert.assertEquals(username, risplit.getPrincipal());
-    Assert.assertEquals(table, risplit.getTableName());
-    Assert.assertEquals(password, risplit.getToken());
-    Assert.assertEquals(auths, risplit.getAuths());
-    Assert.assertEquals(inst.getInstanceName(), risplit.getInstanceName());
-    Assert.assertEquals(isolated, risplit.isIsolatedScan());
-    Assert.assertEquals(localIters, risplit.usesLocalIterators());
-    Assert.assertEquals(fetchColumns, risplit.getFetchedColumns());
-    Assert.assertEquals(level, risplit.getLogLevel());
-  }
-
-  @Test
-  public void testPartialInputSplitDelegationToConfiguration() throws Exception {
-    String user = "testPartialInputSplitUser";
-    PasswordToken password = new PasswordToken("");
-
-    Connector c = inst.getConnector(user, password);
-    c.tableOperations().create("testtable");
-    BatchWriter bw = c.createBatchWriter("testtable", new BatchWriterConfig());
-    for (int i = 0; i < 100; i++) {
-      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
-      bw.addMutation(m);
-    }
-    bw.close();
-
-    Assert.assertEquals(0,
-        MRTester.main(new String[] {user, "", "testtable", "testPartialInputSplitDelegationToConfiguration", EmptySplitsAccumuloInputFormat.class.getName()}));
-    assertNull(e1);
-    assertNull(e2);
-  }
-
-  @Test
-  public void testPartialFailedInputSplitDelegationToConfiguration() throws Exception {
-    String user = "testPartialFailedInputSplit";
-    PasswordToken password = new PasswordToken("");
-
-    Connector c = inst.getConnector(user, password);
-    c.tableOperations().create("testtable");
-    BatchWriter bw = c.createBatchWriter("testtable", new BatchWriterConfig());
-    for (int i = 0; i < 100; i++) {
-      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
-      bw.addMutation(m);
-    }
-    bw.close();
-
-    // We should fail before we even get into the Mapper because we can't make the RecordReader
-    Assert.assertEquals(
-        1,
-        MRTester.main(new String[] {user, "", "testtable", "testPartialFailedInputSplitDelegationToConfiguration",
-            BadPasswordSplitsAccumuloInputFormat.class.getName()}));
-    assertNull(e1);
-    assertNull(e2);
-  }
-
   @Test
   public void testEmptyColumnFamily() throws IOException {
     Job job = Job.getInstance();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
index f1c3ca9..1137916 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormatTest.java
@@ -17,156 +17,33 @@
 package org.apache.accumulo.core.client.mapreduce;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
 
-import java.io.File;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 
-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.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.util.Pair;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TestName;
 
 public class AccumuloMultiTableInputFormatTest {
 
-  private static final String PREFIX = AccumuloMultiTableInputFormatTest.class.getSimpleName();
-  private static final String INSTANCE_NAME = PREFIX + "_mapreduce_instance";
-  private static final String TEST_TABLE_1 = PREFIX + "_mapreduce_table_1";
-  private static final String TEST_TABLE_2 = PREFIX + "_mapreduce_table_2";
-
-  private static AssertionError e1 = null;
-  private static AssertionError e2 = null;
-
-  private static class MRTester extends Configured implements Tool {
-
-    private static class TestMapper extends Mapper<Key,Value,Key,Value> {
-      Key key = null;
-      int count = 0;
-
-      @Override
-      protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
-        try {
-          String tableName = ((RangeInputSplit) context.getInputSplit()).getTableName();
-          if (key != null)
-            assertEquals(key.getRow().toString(), new String(v.get()));
-          assertEquals(new Text(String.format("%s_%09x", tableName, count + 1)), k.getRow());
-          assertEquals(String.format("%s_%09x", tableName, count), new String(v.get()));
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        key = new Key(k);
-        count++;
-      }
-
-      @Override
-      protected void cleanup(Context context) throws IOException, InterruptedException {
-        try {
-          assertEquals(100, count);
-        } catch (AssertionError e) {
-          e2 = e;
-        }
-      }
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 4) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table1> <table2>");
-      }
-
-      String user = args[0];
-      String pass = args[1];
-      String table1 = args[2];
-      String table2 = args[3];
-
-      Job job = Job.getInstance(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormatClass(AccumuloMultiTableInputFormat.class);
-
-      AccumuloMultiTableInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
-
-      InputTableConfig tableConfig1 = new InputTableConfig();
-      InputTableConfig tableConfig2 = new InputTableConfig();
-
-      Map<String,InputTableConfig> configMap = new HashMap<String,InputTableConfig>();
-      configMap.put(table1, tableConfig1);
-      configMap.put(table2, tableConfig2);
-
-      AccumuloMultiTableInputFormat.setInputTableConfigs(job, configMap);
-      AccumuloMultiTableInputFormat.setMockInstance(job, INSTANCE_NAME);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormatClass(NullOutputFormat.class);
-
-      job.setNumReduceTasks(0);
-
-      job.waitForCompletion(true);
-
-      return job.isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
-
-  /**
-   * Generate incrementing counts and attach table name to the key/value so that order and multi-table data can be verified.
-   */
-  @Test
-  public void testMap() throws Exception {
-    Instance mockInstance = new org.apache.accumulo.core.client.mock.MockInstance(INSTANCE_NAME);
-    Connector c = mockInstance.getConnector("root", new PasswordToken(""));
-    c.tableOperations().create(TEST_TABLE_1);
-    c.tableOperations().create(TEST_TABLE_2);
-    BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
-    BatchWriter bw2 = c.createBatchWriter(TEST_TABLE_2, new BatchWriterConfig());
-    for (int i = 0; i < 100; i++) {
-      Mutation t1m = new Mutation(new Text(String.format("%s_%09x", TEST_TABLE_1, i + 1)));
-      t1m.put(new Text(), new Text(), new Value(String.format("%s_%09x", TEST_TABLE_1, i).getBytes()));
-      bw.addMutation(t1m);
-      Mutation t2m = new Mutation(new Text(String.format("%s_%09x", TEST_TABLE_2, i + 1)));
-      t2m.put(new Text(), new Text(), new Value(String.format("%s_%09x", TEST_TABLE_2, i).getBytes()));
-      bw2.addMutation(t2m);
-    }
-    bw.close();
-    bw2.close();
-
-    MRTester.main(new String[] {"root", "", TEST_TABLE_1, TEST_TABLE_2});
-    assertNull(e1);
-    assertNull(e2);
-  }
+  @Rule
+  public TestName testName = new TestName();
 
   /**
    * Verify {@link InputTableConfig} objects get correctly serialized in the JobContext.
    */
   @Test
   public void testInputTableConfigSerialization() throws IOException {
+    String table1 = testName.getMethodName() + "1";
+    String table2 = testName.getMethodName() + "2";
     Job job = Job.getInstance();
 
     InputTableConfig tableConfig = new InputTableConfig().setRanges(Collections.singletonList(new Range("a", "b")))
@@ -174,13 +51,13 @@ public class AccumuloMultiTableInputFormatTest {
         .setIterators(Collections.singletonList(new IteratorSetting(50, "iter1", "iterclass1")));
 
     Map<String,InputTableConfig> configMap = new HashMap<String,InputTableConfig>();
-    configMap.put(TEST_TABLE_1, tableConfig);
-    configMap.put(TEST_TABLE_2, tableConfig);
+    configMap.put(table1, tableConfig);
+    configMap.put(table2, tableConfig);
 
     AccumuloMultiTableInputFormat.setInputTableConfigs(job, configMap);
 
-    assertEquals(tableConfig, AccumuloMultiTableInputFormat.getInputTableConfig(job, TEST_TABLE_1));
-    assertEquals(tableConfig, AccumuloMultiTableInputFormat.getInputTableConfig(job, TEST_TABLE_2));
+    assertEquals(tableConfig, AccumuloMultiTableInputFormat.getInputTableConfig(job, table1));
+    assertEquals(tableConfig, AccumuloMultiTableInputFormat.getInputTableConfig(job, table2));
   }
 
 }


[04/14] accumulo git commit: ACCUMULO-3920 Convert more tests from mock

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/ClientSideIteratorIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/ClientSideIteratorIT.java b/test/src/main/java/org/apache/accumulo/test/ClientSideIteratorIT.java
new file mode 100644
index 0000000..11fb6b0
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/ClientSideIteratorIT.java
@@ -0,0 +1,144 @@
+/*
+ * 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.test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.ClientSideIteratorScanner;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.PartialKey;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.iterators.user.IntersectingIterator;
+import org.apache.accumulo.core.iterators.user.VersioningIterator;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Test;
+
+public class ClientSideIteratorIT extends AccumuloClusterHarness {
+  private List<Key> resultSet1;
+  private List<Key> resultSet2;
+  private List<Key> resultSet3;
+
+  @Before
+  public void setupData() {
+    resultSet1 = new ArrayList<Key>();
+    resultSet1.add(new Key("row1", "colf", "colq", 4l));
+    resultSet1.add(new Key("row1", "colf", "colq", 3l));
+    resultSet2 = new ArrayList<Key>();
+    resultSet2.add(new Key("row1", "colf", "colq", 4l));
+    resultSet2.add(new Key("row1", "colf", "colq", 3l));
+    resultSet2.add(new Key("row1", "colf", "colq", 2l));
+    resultSet2.add(new Key("row1", "colf", "colq", 1l));
+    resultSet3 = new ArrayList<Key>();
+    resultSet3.add(new Key("part1", "", "doc2"));
+    resultSet3.add(new Key("part2", "", "DOC2"));
+  }
+
+  private void checkResults(final Iterable<Entry<Key,Value>> scanner, final List<Key> results, final PartialKey pk) {
+    int i = 0;
+    for (Entry<Key,Value> entry : scanner) {
+      assertTrue(entry.getKey().equals(results.get(i++), pk));
+    }
+    assertEquals(i, results.size());
+  }
+
+  private Connector conn;
+  private String tableName;
+
+  @Before
+  public void setupInstance() throws Exception {
+    conn = getConnector();
+    tableName = getUniqueNames(1)[0];
+  }
+
+  @Test
+  public void testIntersect() throws Exception {
+    conn.tableOperations().create(tableName);
+    BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig());
+    Mutation m = new Mutation("part1");
+    m.put("bar", "doc1", "value");
+    m.put("bar", "doc2", "value");
+    m.put("dog", "doc3", "value");
+    m.put("foo", "doc2", "value");
+    m.put("foo", "doc3", "value");
+    bw.addMutation(m);
+    m = new Mutation("part2");
+    m.put("bar", "DOC1", "value");
+    m.put("bar", "DOC2", "value");
+    m.put("dog", "DOC3", "value");
+    m.put("foo", "DOC2", "value");
+    m.put("foo", "DOC3", "value");
+    bw.addMutation(m);
+    bw.flush();
+
+    final ClientSideIteratorScanner csis = new ClientSideIteratorScanner(conn.createScanner(tableName, new Authorizations()));
+    final IteratorSetting si = new IteratorSetting(10, tableName, IntersectingIterator.class);
+    IntersectingIterator.setColumnFamilies(si, new Text[] {new Text("bar"), new Text("foo")});
+    csis.addScanIterator(si);
+
+    checkResults(csis, resultSet3, PartialKey.ROW_COLFAM_COLQUAL);
+  }
+
+  @Test
+  public void testVersioning() throws Exception {
+    conn.tableOperations().create(tableName);
+    conn.tableOperations().removeProperty(tableName, "table.iterator.scan.vers");
+    conn.tableOperations().removeProperty(tableName, "table.iterator.majc.vers");
+    conn.tableOperations().removeProperty(tableName, "table.iterator.minc.vers");
+    final BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig());
+    Mutation m = new Mutation("row1");
+    m.put("colf", "colq", 1l, "value");
+    m.put("colf", "colq", 2l, "value");
+    bw.addMutation(m);
+    bw.flush();
+    m = new Mutation("row1");
+    m.put("colf", "colq", 3l, "value");
+    m.put("colf", "colq", 4l, "value");
+    bw.addMutation(m);
+    bw.flush();
+
+    final Scanner scanner = conn.createScanner(tableName, new Authorizations());
+
+    final ClientSideIteratorScanner csis = new ClientSideIteratorScanner(scanner);
+    final IteratorSetting si = new IteratorSetting(10, "localvers", VersioningIterator.class);
+    si.addOption("maxVersions", "2");
+    csis.addScanIterator(si);
+
+    checkResults(csis, resultSet1, PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME);
+    checkResults(scanner, resultSet2, PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME);
+
+    csis.fetchColumnFamily(new Text("colf"));
+    checkResults(csis, resultSet1, PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME);
+    csis.clearColumns();
+    csis.fetchColumnFamily(new Text("none"));
+    assertFalse(csis.iterator().hasNext());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
index 621620f..249d88f 100644
--- a/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ShellServerIT.java
@@ -34,6 +34,7 @@ import java.lang.reflect.Constructor;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -62,6 +63,7 @@ import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.file.FileSKVWriter;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.format.Formatter;
 import org.apache.accumulo.harness.SharedMiniClusterBase;
 import org.apache.accumulo.shell.Shell;
 import org.apache.accumulo.test.functional.SlowIterator;
@@ -86,6 +88,7 @@ import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Iterators;
 
 public class ShellServerIT extends SharedMiniClusterBase {
@@ -1083,6 +1086,115 @@ public class ShellServerIT extends SharedMiniClusterBase {
   }
 
   @Test
+  public void formatter() throws Exception {
+    ts.exec("createtable formatter_test", true);
+    ts.exec("table formatter_test", true);
+    ts.exec("insert row cf cq 1234abcd", true);
+    ts.exec("insert row cf1 cq1 9876fedc", true);
+    ts.exec("insert row2 cf cq 13579bdf", true);
+    ts.exec("insert row2 cf1 cq 2468ace", true);
+
+    ArrayList<String> expectedDefault = new ArrayList<>(4);
+    expectedDefault.add("row cf:cq []    1234abcd");
+    expectedDefault.add("row cf1:cq1 []    9876fedc");
+    expectedDefault.add("row2 cf:cq []    13579bdf");
+    expectedDefault.add("row2 cf1:cq []    2468ace");
+    ArrayList<String> actualDefault = new ArrayList<>(4);
+    boolean isFirst = true;
+    for (String s : ts.exec("scan -np", true).split("[\n\r]+")) {
+      if (isFirst) {
+        isFirst = false;
+      } else {
+        actualDefault.add(s);
+      }
+    }
+
+    ArrayList<String> expectedFormatted = new ArrayList<>(4);
+    expectedFormatted.add("row cf:cq []    0x31 0x32 0x33 0x34 0x61 0x62 0x63 0x64");
+    expectedFormatted.add("row cf1:cq1 []    0x39 0x38 0x37 0x36 0x66 0x65 0x64 0x63");
+    expectedFormatted.add("row2 cf:cq []    0x31 0x33 0x35 0x37 0x39 0x62 0x64 0x66");
+    expectedFormatted.add("row2 cf1:cq []    0x32 0x34 0x36 0x38 0x61 0x63 0x65");
+    ts.exec("formatter -t formatter_test -f " + HexFormatter.class.getName(), true);
+    ArrayList<String> actualFormatted = new ArrayList<>(4);
+    isFirst = true;
+    for (String s : ts.exec("scan -np", true).split("[\n\r]+")) {
+      if (isFirst) {
+        isFirst = false;
+      } else {
+        actualFormatted.add(s);
+      }
+    }
+
+    ts.exec("deletetable -f formatter_test", true);
+
+    assertTrue(Iterables.elementsEqual(expectedDefault, new ArrayList<String>(actualDefault)));
+    assertTrue(Iterables.elementsEqual(expectedFormatted, new ArrayList<String>(actualFormatted)));
+  }
+
+  /**
+   * <p>
+   * Simple <code>Formatter</code> that will convert each character in the Value from decimal to hexadecimal. Will automatically skip over characters in the
+   * value which do not fall within the [0-9,a-f] range.
+   * </p>
+   *
+   * <p>
+   * Example: <code>'0'</code> will be displayed as <code>'0x30'</code>
+   * </p>
+   */
+  public static class HexFormatter implements Formatter {
+    private Iterator<Entry<Key,Value>> iter = null;
+    private boolean printTs = false;
+
+    private final static String tab = "\t";
+    private final static String newline = "\n";
+
+    public HexFormatter() {}
+
+    @Override
+    public boolean hasNext() {
+      return this.iter.hasNext();
+    }
+
+    @Override
+    public String next() {
+      final Entry<Key,Value> entry = iter.next();
+
+      String key;
+
+      // Observe the timestamps
+      if (printTs) {
+        key = entry.getKey().toString();
+      } else {
+        key = entry.getKey().toStringNoTime();
+      }
+
+      final Value v = entry.getValue();
+
+      // Approximate how much space we'll need
+      final StringBuilder sb = new StringBuilder(key.length() + v.getSize() * 5);
+
+      sb.append(key).append(tab);
+
+      for (byte b : v.get()) {
+        if ((b >= 48 && b <= 57) || (b >= 97 && b <= 102)) {
+          sb.append(String.format("0x%x ", Integer.valueOf(b)));
+        }
+      }
+
+      return sb.toString().trim() + newline;
+    }
+
+    @Override
+    public void remove() {}
+
+    @Override
+    public void initialize(final Iterable<Entry<Key,Value>> scanner, final boolean printTimestamps) {
+      this.iter = scanner.iterator();
+      this.printTs = printTimestamps;
+    }
+  }
+
+  @Test
   public void extensions() throws Exception {
     String extName = "ExampleShellExtension";
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/examples/simple/dirlist/CountIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/examples/simple/dirlist/CountIT.java b/test/src/main/java/org/apache/accumulo/test/examples/simple/dirlist/CountIT.java
new file mode 100644
index 0000000..8853733
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/examples/simple/dirlist/CountIT.java
@@ -0,0 +1,101 @@
+/*
+ * 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.test.examples.simple.dirlist;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.util.ArrayList;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.cli.BatchWriterOpts;
+import org.apache.accumulo.core.cli.ScannerOpts;
+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.Scanner;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.examples.simple.dirlist.FileCount;
+import org.apache.accumulo.examples.simple.dirlist.FileCount.Opts;
+import org.apache.accumulo.examples.simple.dirlist.Ingest;
+import org.apache.accumulo.examples.simple.dirlist.QueryUtil;
+import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Test;
+
+public class CountIT extends ConfigurableMacBase {
+
+  private Connector conn;
+  private String tableName;
+
+  @Before
+  public void setupInstance() throws Exception {
+    tableName = getUniqueNames(1)[0];
+    conn = getConnector();
+    conn.tableOperations().create(tableName);
+    BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig());
+    ColumnVisibility cv = new ColumnVisibility();
+    // / has 1 dir
+    // /local has 2 dirs 1 file
+    // /local/user1 has 2 files
+    bw.addMutation(Ingest.buildMutation(cv, "/local", true, false, true, 272, 12345, null));
+    bw.addMutation(Ingest.buildMutation(cv, "/local/user1", true, false, true, 272, 12345, null));
+    bw.addMutation(Ingest.buildMutation(cv, "/local/user2", true, false, true, 272, 12345, null));
+    bw.addMutation(Ingest.buildMutation(cv, "/local/file", false, false, false, 1024, 12345, null));
+    bw.addMutation(Ingest.buildMutation(cv, "/local/file", false, false, false, 1024, 23456, null));
+    bw.addMutation(Ingest.buildMutation(cv, "/local/user1/file1", false, false, false, 2024, 12345, null));
+    bw.addMutation(Ingest.buildMutation(cv, "/local/user1/file2", false, false, false, 1028, 23456, null));
+    bw.close();
+  }
+
+  @Test
+  public void test() throws Exception {
+    Scanner scanner = conn.createScanner(tableName, new Authorizations());
+    scanner.fetchColumn(new Text("dir"), new Text("counts"));
+    assertFalse(scanner.iterator().hasNext());
+
+    Opts opts = new Opts();
+    ScannerOpts scanOpts = new ScannerOpts();
+    BatchWriterOpts bwOpts = new BatchWriterOpts();
+    opts.instance = conn.getInstance().getInstanceName();
+    opts.zookeepers = conn.getInstance().getZooKeepers();
+    opts.setTableName(tableName);
+    opts.setPrincipal(conn.whoami());
+    opts.setPassword(new Opts.Password(ROOT_PASSWORD));
+    FileCount fc = new FileCount(opts, scanOpts, bwOpts);
+    fc.run();
+
+    ArrayList<Pair<String,String>> expected = new ArrayList<Pair<String,String>>();
+    expected.add(new Pair<String,String>(QueryUtil.getRow("").toString(), "1,0,3,3"));
+    expected.add(new Pair<String,String>(QueryUtil.getRow("/local").toString(), "2,1,2,3"));
+    expected.add(new Pair<String,String>(QueryUtil.getRow("/local/user1").toString(), "0,2,0,2"));
+    expected.add(new Pair<String,String>(QueryUtil.getRow("/local/user2").toString(), "0,0,0,0"));
+
+    int i = 0;
+    for (Entry<Key,Value> e : scanner) {
+      assertEquals(e.getKey().getRow().toString(), expected.get(i).getFirst());
+      assertEquals(e.getValue().toString(), expected.get(i).getSecond());
+      i++;
+    }
+    assertEquals(i, expected.size());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/examples/simple/filedata/ChunkInputFormatIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/examples/simple/filedata/ChunkInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/examples/simple/filedata/ChunkInputFormatIT.java
new file mode 100644
index 0000000..b2b6aff
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/examples/simple/filedata/ChunkInputFormatIT.java
@@ -0,0 +1,319 @@
+/*
+ * 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.test.examples.simple.filedata;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+
+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.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.examples.simple.filedata.ChunkInputFormat;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+
+public class ChunkInputFormatIT extends AccumuloClusterHarness {
+
+  // track errors in the map reduce job; jobs insert a dummy error for the map and cleanup tasks (to ensure test correctness),
+  // so error tests should check to see if there is at least one error (could be more depending on the test) rather than zero
+  private static Multimap<String,AssertionError> assertionErrors = ArrayListMultimap.create();
+
+  private static final Authorizations AUTHS = new Authorizations("A", "B", "C", "D");
+
+  private static List<Entry<Key,Value>> data;
+  private static List<Entry<Key,Value>> baddata;
+
+  private Connector conn;
+  private String tableName;
+
+  @Before
+  public void setupInstance() throws Exception {
+    conn = getConnector();
+    tableName = getUniqueNames(1)[0];
+    conn.securityOperations().changeUserAuthorizations(conn.whoami(), AUTHS);
+  }
+
+  @BeforeClass
+  public static void setupClass() {
+    System.setProperty("hadoop.tmp.dir", System.getProperty("user.dir") + "/target/hadoop-tmp");
+
+    data = new ArrayList<Entry<Key,Value>>();
+    ChunkInputStreamIT.addData(data, "a", "refs", "ida\0ext", "A&B", "ext");
+    ChunkInputStreamIT.addData(data, "a", "refs", "ida\0name", "A&B", "name");
+    ChunkInputStreamIT.addData(data, "a", "~chunk", 100, 0, "A&B", "asdfjkl;");
+    ChunkInputStreamIT.addData(data, "a", "~chunk", 100, 1, "A&B", "");
+    ChunkInputStreamIT.addData(data, "b", "refs", "ida\0ext", "A&B", "ext");
+    ChunkInputStreamIT.addData(data, "b", "refs", "ida\0name", "A&B", "name");
+    ChunkInputStreamIT.addData(data, "b", "~chunk", 100, 0, "A&B", "qwertyuiop");
+    ChunkInputStreamIT.addData(data, "b", "~chunk", 100, 0, "B&C", "qwertyuiop");
+    ChunkInputStreamIT.addData(data, "b", "~chunk", 100, 1, "A&B", "");
+    ChunkInputStreamIT.addData(data, "b", "~chunk", 100, 1, "B&C", "");
+    ChunkInputStreamIT.addData(data, "b", "~chunk", 100, 1, "D", "");
+    baddata = new ArrayList<Entry<Key,Value>>();
+    ChunkInputStreamIT.addData(baddata, "c", "refs", "ida\0ext", "A&B", "ext");
+    ChunkInputStreamIT.addData(baddata, "c", "refs", "ida\0name", "A&B", "name");
+  }
+
+  public static void entryEquals(Entry<Key,Value> e1, Entry<Key,Value> e2) {
+    assertEquals(e1.getKey(), e2.getKey());
+    assertEquals(e1.getValue(), e2.getValue());
+  }
+
+  public static class CIFTester extends Configured implements Tool {
+    public static class TestMapper extends Mapper<List<Entry<Key,Value>>,InputStream,List<Entry<Key,Value>>,InputStream> {
+      int count = 0;
+
+      @Override
+      protected void map(List<Entry<Key,Value>> key, InputStream value, Context context) throws IOException, InterruptedException {
+        String table = context.getConfiguration().get("MRTester_tableName");
+        assertNotNull(table);
+
+        byte[] b = new byte[20];
+        int read;
+        try {
+          switch (count) {
+            case 0:
+              assertEquals(key.size(), 2);
+              entryEquals(key.get(0), data.get(0));
+              entryEquals(key.get(1), data.get(1));
+              assertEquals(read = value.read(b), 8);
+              assertEquals(new String(b, 0, read), "asdfjkl;");
+              assertEquals(read = value.read(b), -1);
+              break;
+            case 1:
+              assertEquals(key.size(), 2);
+              entryEquals(key.get(0), data.get(4));
+              entryEquals(key.get(1), data.get(5));
+              assertEquals(read = value.read(b), 10);
+              assertEquals(new String(b, 0, read), "qwertyuiop");
+              assertEquals(read = value.read(b), -1);
+              break;
+            default:
+              fail();
+          }
+        } catch (AssertionError e) {
+          assertionErrors.put(table, e);
+        } finally {
+          value.close();
+        }
+        count++;
+      }
+
+      @Override
+      protected void cleanup(Context context) throws IOException, InterruptedException {
+        String table = context.getConfiguration().get("MRTester_tableName");
+        assertNotNull(table);
+
+        try {
+          assertEquals(2, count);
+        } catch (AssertionError e) {
+          assertionErrors.put(table, e);
+        }
+      }
+    }
+
+    public static class TestNoClose extends Mapper<List<Entry<Key,Value>>,InputStream,List<Entry<Key,Value>>,InputStream> {
+      int count = 0;
+
+      @Override
+      protected void map(List<Entry<Key,Value>> key, InputStream value, Context context) throws IOException, InterruptedException {
+        String table = context.getConfiguration().get("MRTester_tableName");
+        assertNotNull(table);
+
+        byte[] b = new byte[5];
+        int read;
+        try {
+          switch (count) {
+            case 0:
+              assertEquals(read = value.read(b), 5);
+              assertEquals(new String(b, 0, read), "asdfj");
+              break;
+            default:
+              fail();
+          }
+        } catch (AssertionError e) {
+          assertionErrors.put(table, e);
+        }
+        count++;
+        try {
+          context.nextKeyValue();
+          fail();
+        } catch (IOException ioe) {
+          assertionErrors.put(table + "_map_ioexception", new AssertionError(toString(), ioe));
+        }
+      }
+    }
+
+    public static class TestBadData extends Mapper<List<Entry<Key,Value>>,InputStream,List<Entry<Key,Value>>,InputStream> {
+      @Override
+      protected void map(List<Entry<Key,Value>> key, InputStream value, Context context) throws IOException, InterruptedException {
+        String table = context.getConfiguration().get("MRTester_tableName");
+        assertNotNull(table);
+
+        byte[] b = new byte[20];
+        try {
+          assertEquals(key.size(), 2);
+          entryEquals(key.get(0), baddata.get(0));
+          entryEquals(key.get(1), baddata.get(1));
+        } catch (AssertionError e) {
+          assertionErrors.put(table, e);
+        }
+        try {
+          assertFalse(value.read(b) > 0);
+          try {
+            fail();
+          } catch (AssertionError e) {
+            assertionErrors.put(table, e);
+          }
+        } catch (Exception e) {
+          // expected, ignore
+        }
+        try {
+          value.close();
+          try {
+            fail();
+          } catch (AssertionError e) {
+            assertionErrors.put(table, e);
+          }
+        } catch (Exception e) {
+          // expected, ignore
+        }
+      }
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+      if (args.length != 2) {
+        throw new IllegalArgumentException("Usage : " + CIFTester.class.getName() + " <table> <mapperClass>");
+      }
+
+      String table = args[0];
+      assertionErrors.put(table, new AssertionError("Dummy"));
+      assertionErrors.put(table + "_map_ioexception", new AssertionError("Dummy_ioexception"));
+      getConf().set("MRTester_tableName", table);
+
+      Job job = Job.getInstance(getConf());
+      job.setJobName(this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
+      job.setJarByClass(this.getClass());
+
+      job.setInputFormatClass(ChunkInputFormat.class);
+
+      ChunkInputFormat.setZooKeeperInstance(job, getCluster().getClientConfig());
+      ChunkInputFormat.setConnectorInfo(job, getAdminPrincipal(), getAdminToken());
+      ChunkInputFormat.setInputTableName(job, table);
+      ChunkInputFormat.setScanAuthorizations(job, AUTHS);
+
+      @SuppressWarnings("unchecked")
+      Class<? extends Mapper<?,?,?,?>> forName = (Class<? extends Mapper<?,?,?,?>>) Class.forName(args[1]);
+      job.setMapperClass(forName);
+      job.setMapOutputKeyClass(Key.class);
+      job.setMapOutputValueClass(Value.class);
+      job.setOutputFormatClass(NullOutputFormat.class);
+
+      job.setNumReduceTasks(0);
+
+      job.waitForCompletion(true);
+
+      return job.isSuccessful() ? 0 : 1;
+    }
+
+    public static int main(String... args) throws Exception {
+      Configuration conf = new Configuration();
+      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
+      return ToolRunner.run(conf, new CIFTester(), args);
+    }
+  }
+
+  @Test
+  public void test() throws Exception {
+    conn.tableOperations().create(tableName);
+    BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    for (Entry<Key,Value> e : data) {
+      Key k = e.getKey();
+      Mutation m = new Mutation(k.getRow());
+      m.put(k.getColumnFamily(), k.getColumnQualifier(), new ColumnVisibility(k.getColumnVisibility()), k.getTimestamp(), e.getValue());
+      bw.addMutation(m);
+    }
+    bw.close();
+
+    assertEquals(0, CIFTester.main(tableName, CIFTester.TestMapper.class.getName()));
+    assertEquals(1, assertionErrors.get(tableName).size());
+  }
+
+  @Test
+  public void testErrorOnNextWithoutClose() throws Exception {
+    conn.tableOperations().create(tableName);
+    BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    for (Entry<Key,Value> e : data) {
+      Key k = e.getKey();
+      Mutation m = new Mutation(k.getRow());
+      m.put(k.getColumnFamily(), k.getColumnQualifier(), new ColumnVisibility(k.getColumnVisibility()), k.getTimestamp(), e.getValue());
+      bw.addMutation(m);
+    }
+    bw.close();
+
+    assertEquals(1, CIFTester.main(tableName, CIFTester.TestNoClose.class.getName()));
+    assertEquals(1, assertionErrors.get(tableName).size());
+    // this should actually exist, in addition to the dummy entry
+    assertEquals(2, assertionErrors.get(tableName + "_map_ioexception").size());
+  }
+
+  @Test
+  public void testInfoWithoutChunks() throws Exception {
+    conn.tableOperations().create(tableName);
+    BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig());
+    for (Entry<Key,Value> e : baddata) {
+      Key k = e.getKey();
+      Mutation m = new Mutation(k.getRow());
+      m.put(k.getColumnFamily(), k.getColumnQualifier(), new ColumnVisibility(k.getColumnVisibility()), k.getTimestamp(), e.getValue());
+      bw.addMutation(m);
+    }
+    bw.close();
+
+    assertEquals(0, CIFTester.main(tableName, CIFTester.TestBadData.class.getName()));
+    assertEquals(1, assertionErrors.get(tableName).size());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/examples/simple/filedata/ChunkInputStreamIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/examples/simple/filedata/ChunkInputStreamIT.java b/test/src/main/java/org/apache/accumulo/test/examples/simple/filedata/ChunkInputStreamIT.java
new file mode 100644
index 0000000..47db6dd
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/examples/simple/filedata/ChunkInputStreamIT.java
@@ -0,0 +1,174 @@
+/*
+ * 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.test.examples.simple.filedata;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+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.BatchWriter;
+import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.Connector;
+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.data.Key;
+import org.apache.accumulo.core.data.KeyValue;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.ColumnVisibility;
+import org.apache.accumulo.core.util.PeekingIterator;
+import org.apache.accumulo.examples.simple.filedata.ChunkInputStream;
+import org.apache.accumulo.examples.simple.filedata.FileDataIngest;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Test;
+
+public class ChunkInputStreamIT extends AccumuloClusterHarness {
+
+  private static final Authorizations AUTHS = new Authorizations("A", "B", "C", "D");
+
+  private Connector conn;
+  private String tableName;
+  private List<Entry<Key,Value>> data;
+  private List<Entry<Key,Value>> baddata;
+  private List<Entry<Key,Value>> multidata;
+
+  @Before
+  public void setupInstance() throws Exception {
+    conn = getConnector();
+    tableName = getUniqueNames(1)[0];
+    conn.securityOperations().changeUserAuthorizations(conn.whoami(), AUTHS);
+  }
+
+  @Before
+  public void setupData() {
+    data = new ArrayList<Entry<Key,Value>>();
+    addData(data, "a", "refs", "id\0ext", "A&B", "ext");
+    addData(data, "a", "refs", "id\0name", "A&B", "name");
+    addData(data, "a", "~chunk", 100, 0, "A&B", "asdfjkl;");
+    addData(data, "a", "~chunk", 100, 1, "A&B", "");
+    addData(data, "b", "refs", "id\0ext", "A&B", "ext");
+    addData(data, "b", "refs", "id\0name", "A&B", "name");
+    addData(data, "b", "~chunk", 100, 0, "A&B", "qwertyuiop");
+    addData(data, "b", "~chunk", 100, 0, "B&C", "qwertyuiop");
+    addData(data, "b", "~chunk", 100, 1, "A&B", "");
+    addData(data, "b", "~chunk", 100, 1, "B&C", "");
+    addData(data, "b", "~chunk", 100, 1, "D", "");
+    addData(data, "c", "~chunk", 100, 0, "A&B", "asdfjkl;");
+    addData(data, "c", "~chunk", 100, 1, "A&B", "asdfjkl;");
+    addData(data, "c", "~chunk", 100, 2, "A&B", "");
+    addData(data, "d", "~chunk", 100, 0, "A&B", "");
+    addData(data, "e", "~chunk", 100, 0, "A&B", "asdfjkl;");
+    addData(data, "e", "~chunk", 100, 1, "A&B", "");
+    baddata = new ArrayList<Entry<Key,Value>>();
+    addData(baddata, "a", "~chunk", 100, 0, "A", "asdfjkl;");
+    addData(baddata, "b", "~chunk", 100, 0, "B", "asdfjkl;");
+    addData(baddata, "b", "~chunk", 100, 2, "C", "");
+    addData(baddata, "c", "~chunk", 100, 0, "D", "asdfjkl;");
+    addData(baddata, "c", "~chunk", 100, 2, "E", "");
+    addData(baddata, "d", "~chunk", 100, 0, "F", "asdfjkl;");
+    addData(baddata, "d", "~chunk", 100, 1, "G", "");
+    addData(baddata, "d", "~zzzzz", "colq", "H", "");
+    addData(baddata, "e", "~chunk", 100, 0, "I", "asdfjkl;");
+    addData(baddata, "e", "~chunk", 100, 1, "J", "");
+    addData(baddata, "e", "~chunk", 100, 2, "I", "asdfjkl;");
+    addData(baddata, "f", "~chunk", 100, 2, "K", "asdfjkl;");
+    addData(baddata, "g", "~chunk", 100, 0, "L", "");
+    multidata = new ArrayList<Entry<Key,Value>>();
+    addData(multidata, "a", "~chunk", 100, 0, "A&B", "asdfjkl;");
+    addData(multidata, "a", "~chunk", 100, 1, "A&B", "");
+    addData(multidata, "a", "~chunk", 200, 0, "B&C", "asdfjkl;");
+    addData(multidata, "b", "~chunk", 100, 0, "A&B", "asdfjkl;");
+    addData(multidata, "b", "~chunk", 200, 0, "B&C", "asdfjkl;");
+    addData(multidata, "b", "~chunk", 200, 1, "B&C", "asdfjkl;");
+    addData(multidata, "c", "~chunk", 100, 0, "A&B", "asdfjkl;");
+    addData(multidata, "c", "~chunk", 100, 1, "B&C", "");
+  }
+
+  static void addData(List<Entry<Key,Value>> data, String row, String cf, String cq, String vis, String value) {
+    data.add(new KeyValue(new Key(new Text(row), new Text(cf), new Text(cq), new Text(vis)), value.getBytes()));
+  }
+
+  static void addData(List<Entry<Key,Value>> data, String row, String cf, int chunkSize, int chunkCount, String vis, String value) {
+    Text chunkCQ = new Text(FileDataIngest.intToBytes(chunkSize));
+    chunkCQ.append(FileDataIngest.intToBytes(chunkCount), 0, 4);
+    data.add(new KeyValue(new Key(new Text(row), new Text(cf), chunkCQ, new Text(vis)), value.getBytes()));
+  }
+
+  @Test
+  public void testWithAccumulo() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException, IOException {
+    conn.tableOperations().create(tableName);
+    BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    for (Entry<Key,Value> e : data) {
+      Key k = e.getKey();
+      Mutation m = new Mutation(k.getRow());
+      m.put(k.getColumnFamily(), k.getColumnQualifier(), new ColumnVisibility(k.getColumnVisibility()), e.getValue());
+      bw.addMutation(m);
+    }
+    bw.close();
+
+    Scanner scan = conn.createScanner(tableName, AUTHS);
+
+    ChunkInputStream cis = new ChunkInputStream();
+    byte[] b = new byte[20];
+    int read;
+    PeekingIterator<Entry<Key,Value>> pi = new PeekingIterator<Entry<Key,Value>>(scan.iterator());
+
+    cis.setSource(pi);
+    assertEquals(read = cis.read(b), 8);
+    assertEquals(new String(b, 0, read), "asdfjkl;");
+    assertEquals(read = cis.read(b), -1);
+
+    cis.setSource(pi);
+    assertEquals(read = cis.read(b), 10);
+    assertEquals(new String(b, 0, read), "qwertyuiop");
+    assertEquals(read = cis.read(b), -1);
+    assertEquals(cis.getVisibilities().toString(), "[A&B, B&C, D]");
+    cis.close();
+
+    cis.setSource(pi);
+    assertEquals(read = cis.read(b), 16);
+    assertEquals(new String(b, 0, read), "asdfjkl;asdfjkl;");
+    assertEquals(read = cis.read(b), -1);
+    assertEquals(cis.getVisibilities().toString(), "[A&B]");
+    cis.close();
+
+    cis.setSource(pi);
+    assertEquals(read = cis.read(b), -1);
+    cis.close();
+
+    cis.setSource(pi);
+    assertEquals(read = cis.read(b), 8);
+    assertEquals(new String(b, 0, read), "asdfjkl;");
+    assertEquals(read = cis.read(b), -1);
+    cis.close();
+
+    assertFalse(pi.hasNext());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java
deleted file mode 100644
index 1d38126..0000000
--- a/test/src/main/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.test.functional;
-
-import static java.lang.System.currentTimeMillis;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.TreeSet;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.ClientConfiguration;
-import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
-import org.apache.accumulo.core.client.mapreduce.impl.BatchInputSplit;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.DefaultConfiguration;
-import org.apache.accumulo.core.conf.Property;
-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.harness.AccumuloClusterHarness;
-import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.junit.Before;
-import org.junit.Test;
-
-import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
-
-public class AccumuloInputFormatIT extends AccumuloClusterHarness {
-
-  AccumuloInputFormat inputFormat;
-
-  @Override
-  protected int defaultTimeoutSeconds() {
-    return 4 * 60;
-  }
-
-  @Override
-  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
-  }
-
-  @Before
-  public void before() {
-    inputFormat = new AccumuloInputFormat();
-  }
-
-  /**
-   * Tests several different paths through the getSplits() method by setting different properties and verifying the results.
-   */
-  @Test
-  public void testGetSplits() throws Exception {
-    Connector conn = getConnector();
-    String table = getUniqueNames(1)[0];
-    conn.tableOperations().create(table);
-    insertData(table, currentTimeMillis());
-
-    ClientConfiguration clientConf = cluster.getClientConfig();
-    AccumuloConfiguration clusterClientConf = new ConfigurationCopy(new DefaultConfiguration());
-
-    // Pass SSL and CredentialProvider options into the ClientConfiguration given to AccumuloInputFormat
-    boolean sslEnabled = Boolean.valueOf(clusterClientConf.get(Property.INSTANCE_RPC_SSL_ENABLED));
-    if (sslEnabled) {
-      ClientProperty[] sslProperties = new ClientProperty[] {ClientProperty.INSTANCE_RPC_SSL_ENABLED, ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH,
-          ClientProperty.RPC_SSL_KEYSTORE_PATH, ClientProperty.RPC_SSL_KEYSTORE_TYPE, ClientProperty.RPC_SSL_KEYSTORE_PASSWORD,
-          ClientProperty.RPC_SSL_TRUSTSTORE_PATH, ClientProperty.RPC_SSL_TRUSTSTORE_TYPE, ClientProperty.RPC_SSL_TRUSTSTORE_PASSWORD,
-          ClientProperty.RPC_USE_JSSE, ClientProperty.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS};
-
-      for (ClientProperty prop : sslProperties) {
-        // The default property is returned if it's not in the ClientConfiguration so we don't have to check if the value is actually defined
-        clientConf.setProperty(prop, clusterClientConf.get(prop.getKey()));
-      }
-    }
-
-    Job job = Job.getInstance();
-    AccumuloInputFormat.setInputTableName(job, table);
-    AccumuloInputFormat.setZooKeeperInstance(job, clientConf);
-    AccumuloInputFormat.setConnectorInfo(job, getAdminPrincipal(), getAdminToken());
-
-    // split table
-    TreeSet<Text> splitsToAdd = new TreeSet<Text>();
-    for (int i = 0; i < 10000; i += 1000)
-      splitsToAdd.add(new Text(String.format("%09d", i)));
-    conn.tableOperations().addSplits(table, splitsToAdd);
-    sleepUninterruptibly(500, TimeUnit.MILLISECONDS); // wait for splits to be propagated
-
-    // get splits without setting any range
-    Collection<Text> actualSplits = conn.tableOperations().listSplits(table);
-    List<InputSplit> splits = inputFormat.getSplits(job);
-    assertEquals(actualSplits.size() + 1, splits.size()); // No ranges set on the job so it'll start with -inf
-
-    // set ranges and get splits
-    List<Range> ranges = new ArrayList<Range>();
-    for (Text text : actualSplits)
-      ranges.add(new Range(text));
-    AccumuloInputFormat.setRanges(job, ranges);
-    splits = inputFormat.getSplits(job);
-    assertEquals(actualSplits.size(), splits.size());
-
-    // offline mode
-    AccumuloInputFormat.setOfflineTableScan(job, true);
-    try {
-      inputFormat.getSplits(job);
-      fail("An exception should have been thrown");
-    } catch (IOException e) {}
-
-    conn.tableOperations().offline(table, true);
-    splits = inputFormat.getSplits(job);
-    assertEquals(actualSplits.size(), splits.size());
-
-    // auto adjust ranges
-    ranges = new ArrayList<Range>();
-    for (int i = 0; i < 5; i++)
-      // overlapping ranges
-      ranges.add(new Range(String.format("%09d", i), String.format("%09d", i + 2)));
-    AccumuloInputFormat.setRanges(job, ranges);
-    splits = inputFormat.getSplits(job);
-    assertEquals(2, splits.size());
-
-    AccumuloInputFormat.setAutoAdjustRanges(job, false);
-    splits = inputFormat.getSplits(job);
-    assertEquals(ranges.size(), splits.size());
-
-    // BatchScan not available for offline scans
-    AccumuloInputFormat.setBatchScan(job, true);
-    // Reset auto-adjust ranges too
-    AccumuloInputFormat.setAutoAdjustRanges(job, true);
-
-    AccumuloInputFormat.setOfflineTableScan(job, true);
-    try {
-      inputFormat.getSplits(job);
-      fail("An exception should have been thrown");
-    } catch (IllegalArgumentException e) {}
-
-    conn.tableOperations().online(table, true);
-    AccumuloInputFormat.setOfflineTableScan(job, false);
-
-    // test for resumption of success
-    splits = inputFormat.getSplits(job);
-    assertEquals(2, splits.size());
-
-    // BatchScan not available with isolated iterators
-    AccumuloInputFormat.setScanIsolation(job, true);
-    try {
-      inputFormat.getSplits(job);
-      fail("An exception should have been thrown");
-    } catch (IllegalArgumentException e) {}
-    AccumuloInputFormat.setScanIsolation(job, false);
-
-    // test for resumption of success
-    splits = inputFormat.getSplits(job);
-    assertEquals(2, splits.size());
-
-    // BatchScan not available with local iterators
-    AccumuloInputFormat.setLocalIterators(job, true);
-    try {
-      inputFormat.getSplits(job);
-      fail("An exception should have been thrown");
-    } catch (IllegalArgumentException e) {}
-    AccumuloInputFormat.setLocalIterators(job, false);
-
-    // Check we are getting back correct type pf split
-    conn.tableOperations().online(table);
-    splits = inputFormat.getSplits(job);
-    for (InputSplit split : splits)
-      assert (split instanceof BatchInputSplit);
-
-    // We should divide along the tablet lines similar to when using `setAutoAdjustRanges(job, true)`
-    assertEquals(2, splits.size());
-  }
-
-  private void insertData(String tableName, long ts) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    BatchWriter bw = getConnector().createBatchWriter(tableName, null);
-
-    for (int i = 0; i < 10000; i++) {
-      String row = String.format("%09d", i);
-
-      Mutation m = new Mutation(new Text(row));
-      m.put(new Text("cf1"), new Text("cq1"), ts, new Value(("" + i).getBytes()));
-      bw.addMutation(m);
-    }
-    bw.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/functional/MergeIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MergeIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MergeIT.java
index 9e3e8b6..4652c33 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/MergeIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/MergeIT.java
@@ -27,18 +27,27 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 
 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.Scanner;
 import org.apache.accumulo.core.client.admin.NewTableConfiguration;
 import org.apache.accumulo.core.client.admin.TimeType;
 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.data.impl.KeyExtent;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.Merge;
 import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.server.util.TabletIterator;
+import org.apache.accumulo.server.util.TabletIterator.TabletDeletedException;
 import org.apache.hadoop.io.Text;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 
 public class MergeIT extends AccumuloClusterHarness {
 
@@ -191,4 +200,73 @@ public class MergeIT extends AccumuloClusterHarness {
 
   }
 
+  @Rule
+  public ExpectedException exception = ExpectedException.none();
+
+  private static class TestTabletIterator extends TabletIterator {
+
+    private final Connector conn;
+    private final String metadataTableName;
+
+    public TestTabletIterator(Connector conn, String metadataTableName) throws Exception {
+      super(conn.createScanner(metadataTableName, Authorizations.EMPTY), MetadataSchema.TabletsSection.getRange(), true, true);
+      this.conn = conn;
+      this.metadataTableName = metadataTableName;
+    }
+
+    @Override
+    protected void resetScanner() {
+      try {
+        Scanner ds = conn.createScanner(metadataTableName, Authorizations.EMPTY);
+        Text tablet = new KeyExtent(new Text("0"), new Text("m"), null).getMetadataEntry();
+        ds.setRange(new Range(tablet, true, tablet, true));
+
+        Mutation m = new Mutation(tablet);
+
+        BatchWriter bw = conn.createBatchWriter(metadataTableName, new BatchWriterConfig());
+        for (Entry<Key,Value> entry : ds) {
+          Key k = entry.getKey();
+          m.putDelete(k.getColumnFamily(), k.getColumnQualifier(), k.getTimestamp());
+        }
+
+        bw.addMutation(m);
+
+        bw.close();
+
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+
+      super.resetScanner();
+    }
+
+  }
+
+  // simulate a merge happening while iterating over tablets
+  @Test
+  public void testMerge() throws Exception {
+    // create a fake metadata table
+    String metadataTableName = getUniqueNames(1)[0];
+    getConnector().tableOperations().create(metadataTableName);
+
+    KeyExtent ke1 = new KeyExtent(new Text("0"), new Text("m"), null);
+    Mutation mut1 = ke1.getPrevRowUpdateMutation();
+    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(mut1, new Value("/d1".getBytes()));
+
+    KeyExtent ke2 = new KeyExtent(new Text("0"), null, null);
+    Mutation mut2 = ke2.getPrevRowUpdateMutation();
+    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(mut2, new Value("/d2".getBytes()));
+
+    BatchWriter bw1 = getConnector().createBatchWriter(metadataTableName, new BatchWriterConfig());
+    bw1.addMutation(mut1);
+    bw1.addMutation(mut2);
+    bw1.close();
+
+    TestTabletIterator tabIter = new TestTabletIterator(getConnector(), metadataTableName);
+
+    exception.expect(TabletDeletedException.class);
+    while (tabIter.hasNext()) {
+      tabIter.next();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/gc/replication/CloseWriteAheadLogReferencesIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/gc/replication/CloseWriteAheadLogReferencesIT.java b/test/src/main/java/org/apache/accumulo/test/gc/replication/CloseWriteAheadLogReferencesIT.java
new file mode 100644
index 0000000..6475e11
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/gc/replication/CloseWriteAheadLogReferencesIT.java
@@ -0,0 +1,184 @@
+package org.apache.accumulo.test.gc.replication;
+
+/*
+ * 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.
+ */
+
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.Set;
+
+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.Scanner;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.conf.SiteConfiguration;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
+import org.apache.accumulo.core.protobuf.ProtobufUtil;
+import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
+import org.apache.accumulo.core.replication.ReplicationTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.gc.replication.CloseWriteAheadLogReferences;
+import org.apache.accumulo.server.AccumuloServerContext;
+import org.apache.accumulo.server.conf.ServerConfigurationFactory;
+import org.apache.accumulo.server.replication.StatusUtil;
+import org.apache.accumulo.server.replication.proto.Replication.Status;
+import org.apache.accumulo.test.functional.ConfigurableMacBase;
+import org.apache.hadoop.io.Text;
+import org.easymock.EasyMock;
+import org.easymock.IAnswer;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.Iterables;
+
+public class CloseWriteAheadLogReferencesIT extends ConfigurableMacBase {
+
+  private WrappedCloseWriteAheadLogReferences refs;
+  private Connector conn;
+
+  private static class WrappedCloseWriteAheadLogReferences extends CloseWriteAheadLogReferences {
+    public WrappedCloseWriteAheadLogReferences(AccumuloServerContext context) {
+      super(context);
+    }
+
+    @Override
+    protected long updateReplicationEntries(Connector conn, Set<String> closedWals) {
+      return super.updateReplicationEntries(conn, closedWals);
+    }
+  }
+
+  @Before
+  public void setupInstance() throws Exception {
+    conn = getConnector();
+    conn.securityOperations().grantTablePermission(conn.whoami(), ReplicationTable.NAME, TablePermission.WRITE);
+    conn.securityOperations().grantTablePermission(conn.whoami(), MetadataTable.NAME, TablePermission.WRITE);
+    ReplicationTable.setOnline(conn);
+  }
+
+  @Before
+  public void setupEasyMockStuff() {
+    Instance mockInst = createMock(Instance.class);
+    SiteConfiguration siteConfig = EasyMock.createMock(SiteConfiguration.class);
+    expect(mockInst.getInstanceID()).andReturn(testName.getMethodName()).anyTimes();
+    expect(mockInst.getZooKeepers()).andReturn("localhost").anyTimes();
+    expect(mockInst.getZooKeepersSessionTimeOut()).andReturn(30000).anyTimes();
+    final AccumuloConfiguration systemConf = new ConfigurationCopy(new HashMap<String,String>());
+    ServerConfigurationFactory factory = createMock(ServerConfigurationFactory.class);
+    expect(factory.getConfiguration()).andReturn(systemConf).anyTimes();
+    expect(factory.getInstance()).andReturn(mockInst).anyTimes();
+    expect(factory.getSiteConfiguration()).andReturn(siteConfig).anyTimes();
+
+    // Just make the SiteConfiguration delegate to our AccumuloConfiguration
+    // Presently, we only need get(Property) and iterator().
+    EasyMock.expect(siteConfig.get(EasyMock.anyObject(Property.class))).andAnswer(new IAnswer<String>() {
+      @Override
+      public String answer() {
+        Object[] args = EasyMock.getCurrentArguments();
+        return systemConf.get((Property) args[0]);
+      }
+    }).anyTimes();
+    EasyMock.expect(siteConfig.getBoolean(EasyMock.anyObject(Property.class))).andAnswer(new IAnswer<Boolean>() {
+      @Override
+      public Boolean answer() {
+        Object[] args = EasyMock.getCurrentArguments();
+        return systemConf.getBoolean((Property) args[0]);
+      }
+    }).anyTimes();
+
+    EasyMock.expect(siteConfig.iterator()).andAnswer(new IAnswer<Iterator<Entry<String,String>>>() {
+      @Override
+      public Iterator<Entry<String,String>> answer() {
+        return systemConf.iterator();
+      }
+    }).anyTimes();
+
+    replay(mockInst, factory, siteConfig);
+    refs = new WrappedCloseWriteAheadLogReferences(new AccumuloServerContext(factory));
+  }
+
+  @Test
+  public void unclosedWalsLeaveStatusOpen() throws Exception {
+    Set<String> wals = Collections.emptySet();
+    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+    Mutation m = new Mutation(ReplicationSection.getRowPrefix() + "file:/accumulo/wal/tserver+port/12345");
+    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
+    bw.addMutation(m);
+    bw.close();
+
+    refs.updateReplicationEntries(conn, wals);
+
+    Scanner s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    s.fetchColumnFamily(ReplicationSection.COLF);
+    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
+    Status status = Status.parseFrom(entry.getValue().get());
+    Assert.assertFalse(status.getClosed());
+  }
+
+  @Test
+  public void closedWalsUpdateStatus() throws Exception {
+    String file = "file:/accumulo/wal/tserver+port/12345";
+    Set<String> wals = Collections.singleton(file);
+    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+    Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
+    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
+    bw.addMutation(m);
+    bw.close();
+
+    refs.updateReplicationEntries(conn, wals);
+
+    Scanner s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    s.fetchColumnFamily(ReplicationSection.COLF);
+    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
+    Status status = Status.parseFrom(entry.getValue().get());
+    Assert.assertTrue(status.getClosed());
+  }
+
+  @Test
+  public void partiallyReplicatedReferencedWalsAreNotClosed() throws Exception {
+    String file = "file:/accumulo/wal/tserver+port/12345";
+    Set<String> wals = Collections.singleton(file);
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    Mutation m = new Mutation(file);
+    StatusSection.add(m, new Text("1"), ProtobufUtil.toValue(StatusUtil.ingestedUntil(1000)));
+    bw.addMutation(m);
+    bw.close();
+
+    refs.updateReplicationEntries(conn, wals);
+
+    Scanner s = ReplicationTable.getScanner(conn);
+    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
+    Status status = Status.parseFrom(entry.getValue().get());
+    Assert.assertFalse(status.getClosed());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java
new file mode 100644
index 0000000..7a4223d
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java
@@ -0,0 +1,204 @@
+/*
+ * 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.test.mapred;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+
+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.mapred.AccumuloFileOutputFormat;
+import org.apache.accumulo.core.client.mapred.AccumuloInputFormat;
+import org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.lib.IdentityMapper;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AccumuloFileOutputFormatIT extends AccumuloClusterHarness {
+  private static final Logger log = LoggerFactory.getLogger(AccumuloFileOutputFormatIT.class);
+  private static final int JOB_VISIBILITY_CACHE_SIZE = 3000;
+  private static final String PREFIX = AccumuloFileOutputFormatIT.class.getSimpleName();
+  private static final String BAD_TABLE = PREFIX + "_mapred_bad_table";
+  private static final String TEST_TABLE = PREFIX + "_mapred_test_table";
+  private static final String EMPTY_TABLE = PREFIX + "_mapred_empty_table";
+
+  private static AssertionError e1 = null;
+  private static AssertionError e2 = null;
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
+
+  @Test
+  public void testEmptyWrite() throws Exception {
+    getConnector().tableOperations().create(EMPTY_TABLE);
+    handleWriteTests(false);
+  }
+
+  @Test
+  public void testRealWrite() throws Exception {
+    Connector c = getConnector();
+    c.tableOperations().create(TEST_TABLE);
+    BatchWriter bw = c.createBatchWriter(TEST_TABLE, new BatchWriterConfig());
+    Mutation m = new Mutation("Key");
+    m.put("", "", "");
+    bw.addMutation(m);
+    bw.close();
+    handleWriteTests(true);
+  }
+
+  private static class MRTester extends Configured implements Tool {
+    private static class BadKeyMapper implements Mapper<Key,Value,Key,Value> {
+
+      int index = 0;
+
+      @Override
+      public void map(Key key, Value value, OutputCollector<Key,Value> output, Reporter reporter) throws IOException {
+        try {
+          try {
+            output.collect(key, value);
+            if (index == 2)
+              fail();
+          } catch (Exception e) {
+            log.error(e.toString(), e);
+            assertEquals(2, index);
+          }
+        } catch (AssertionError e) {
+          e1 = e;
+        }
+        index++;
+      }
+
+      @Override
+      public void configure(JobConf job) {}
+
+      @Override
+      public void close() throws IOException {
+        try {
+          assertEquals(2, index);
+        } catch (AssertionError e) {
+          e2 = e;
+        }
+      }
+
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+      if (args.length != 2) {
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <table> <outputfile>");
+      }
+
+      String table = args[0];
+
+      JobConf job = new JobConf(getConf());
+      job.setJarByClass(this.getClass());
+      ConfiguratorBase.setVisibilityCacheSize(job, JOB_VISIBILITY_CACHE_SIZE);
+
+      job.setInputFormat(AccumuloInputFormat.class);
+
+      AccumuloInputFormat.setZooKeeperInstance(job, getCluster().getClientConfig());
+      AccumuloInputFormat.setConnectorInfo(job, getAdminPrincipal(), getAdminToken());
+      AccumuloInputFormat.setInputTableName(job, table);
+      AccumuloFileOutputFormat.setOutputPath(job, new Path(args[1]));
+
+      job.setMapperClass(BAD_TABLE.equals(table) ? BadKeyMapper.class : IdentityMapper.class);
+      job.setMapOutputKeyClass(Key.class);
+      job.setMapOutputValueClass(Value.class);
+      job.setOutputFormat(AccumuloFileOutputFormat.class);
+
+      job.setNumReduceTasks(0);
+
+      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
+    }
+
+    public static void main(String[] args) throws Exception {
+      Configuration conf = new Configuration();
+      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
+      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
+    }
+  }
+
+  private void handleWriteTests(boolean content) throws Exception {
+    File f = folder.newFile(testName.getMethodName());
+    if (f.delete()) {
+      log.debug("Deleted {}", f);
+    }
+    MRTester.main(new String[] {content ? TEST_TABLE : EMPTY_TABLE, f.getAbsolutePath()});
+
+    assertTrue(f.exists());
+    File[] files = f.listFiles(new FileFilter() {
+      @Override
+      public boolean accept(File file) {
+        return file.getName().startsWith("part-m-");
+      }
+    });
+    assertNotNull(files);
+    if (content) {
+      assertEquals(1, files.length);
+      assertTrue(files[0].exists());
+    } else {
+      assertEquals(0, files.length);
+    }
+  }
+
+  @Test
+  public void writeBadVisibility() throws Exception {
+    Connector c = getConnector();
+    c.tableOperations().create(BAD_TABLE);
+    BatchWriter bw = c.createBatchWriter(BAD_TABLE, new BatchWriterConfig());
+    Mutation m = new Mutation("r1");
+    m.put("cf1", "cq1", "A&B");
+    m.put("cf1", "cq1", "A&B");
+    m.put("cf1", "cq2", "A&");
+    bw.addMutation(m);
+    bw.close();
+    File f = folder.newFile(testName.getMethodName());
+    if (f.delete()) {
+      log.debug("Deleted {}", f);
+    }
+    MRTester.main(new String[] {BAD_TABLE, f.getAbsolutePath()});
+    assertNull(e1);
+    assertNull(e2);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloInputFormatIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloInputFormatIT.java
new file mode 100644
index 0000000..2cef382
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloInputFormatIT.java
@@ -0,0 +1,195 @@
+/*
+ * 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.test.mapred;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+
+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.mapred.AccumuloInputFormat;
+import org.apache.accumulo.core.client.mapred.RangeInputSplit;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.Pair;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.lib.NullOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.log4j.Level;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class AccumuloInputFormatIT extends AccumuloClusterHarness {
+
+  @BeforeClass
+  public static void setupClass() {
+    System.setProperty("hadoop.tmp.dir", System.getProperty("user.dir") + "/target/hadoop-tmp");
+  }
+
+  private static AssertionError e1 = null;
+  private static AssertionError e2 = null;
+
+  private static class MRTester extends Configured implements Tool {
+    private static class TestMapper implements Mapper<Key,Value,Key,Value> {
+      Key key = null;
+      int count = 0;
+
+      @Override
+      public void map(Key k, Value v, OutputCollector<Key,Value> output, Reporter reporter) throws IOException {
+        try {
+          if (key != null)
+            assertEquals(key.getRow().toString(), new String(v.get()));
+          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
+          assertEquals(new String(v.get()), String.format("%09x", count));
+        } catch (AssertionError e) {
+          e1 = e;
+        }
+        key = new Key(k);
+        count++;
+      }
+
+      @Override
+      public void configure(JobConf job) {}
+
+      @Override
+      public void close() throws IOException {
+        try {
+          assertEquals(100, count);
+        } catch (AssertionError e) {
+          e2 = e;
+        }
+      }
+
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+      if (args.length != 1) {
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <table>");
+      }
+
+      String table = args[0];
+
+      JobConf job = new JobConf(getConf());
+      job.setJarByClass(this.getClass());
+
+      job.setInputFormat(AccumuloInputFormat.class);
+
+      AccumuloInputFormat.setConnectorInfo(job, getAdminPrincipal(), getAdminToken());
+      AccumuloInputFormat.setInputTableName(job, table);
+      AccumuloInputFormat.setZooKeeperInstance(job, getCluster().getClientConfig());
+
+      job.setMapperClass(TestMapper.class);
+      job.setMapOutputKeyClass(Key.class);
+      job.setMapOutputValueClass(Value.class);
+      job.setOutputFormat(NullOutputFormat.class);
+
+      job.setNumReduceTasks(0);
+
+      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
+    }
+
+    public static void main(String... args) throws Exception {
+      Configuration conf = new Configuration();
+      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
+      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
+    }
+  }
+
+  @Test
+  public void testMap() throws Exception {
+    String table = getUniqueNames(1)[0];
+    Connector c = getConnector();
+    c.tableOperations().create(table);
+    BatchWriter bw = c.createBatchWriter(table, new BatchWriterConfig());
+    for (int i = 0; i < 100; i++) {
+      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
+      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
+      bw.addMutation(m);
+    }
+    bw.close();
+
+    MRTester.main(table);
+    assertNull(e1);
+    assertNull(e2);
+  }
+
+  @Test
+  public void testCorrectRangeInputSplits() throws Exception {
+    JobConf job = new JobConf();
+
+    String table = getUniqueNames(1)[0];
+    Authorizations auths = new Authorizations("foo");
+    Collection<Pair<Text,Text>> fetchColumns = Collections.singleton(new Pair<Text,Text>(new Text("foo"), new Text("bar")));
+    boolean isolated = true, localIters = true;
+    Level level = Level.WARN;
+
+    Connector connector = getConnector();
+    connector.tableOperations().create(table);
+
+    AccumuloInputFormat.setConnectorInfo(job, getAdminPrincipal(), getAdminToken());
+    AccumuloInputFormat.setInputTableName(job, table);
+    AccumuloInputFormat.setScanAuthorizations(job, auths);
+    AccumuloInputFormat.setZooKeeperInstance(job, getCluster().getClientConfig());
+    AccumuloInputFormat.setScanIsolation(job, isolated);
+    AccumuloInputFormat.setLocalIterators(job, localIters);
+    AccumuloInputFormat.fetchColumns(job, fetchColumns);
+    AccumuloInputFormat.setLogLevel(job, level);
+
+    AccumuloInputFormat aif = new AccumuloInputFormat();
+
+    InputSplit[] splits = aif.getSplits(job, 1);
+
+    Assert.assertEquals(1, splits.length);
+
+    InputSplit split = splits[0];
+
+    Assert.assertEquals(RangeInputSplit.class, split.getClass());
+
+    RangeInputSplit risplit = (RangeInputSplit) split;
+
+    Assert.assertEquals(getAdminPrincipal(), risplit.getPrincipal());
+    Assert.assertEquals(table, risplit.getTableName());
+    Assert.assertEquals(getAdminToken(), risplit.getToken());
+    Assert.assertEquals(auths, risplit.getAuths());
+    Assert.assertEquals(getConnector().getInstance().getInstanceName(), risplit.getInstanceName());
+    Assert.assertEquals(isolated, risplit.isIsolatedScan());
+    Assert.assertEquals(localIters, risplit.usesLocalIterators());
+    Assert.assertEquals(fetchColumns, risplit.getFetchedColumns());
+    Assert.assertEquals(level, risplit.getLogLevel());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloMultiTableInputFormatIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloMultiTableInputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloMultiTableInputFormatIT.java
new file mode 100644
index 0000000..35ba7bc
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloMultiTableInputFormatIT.java
@@ -0,0 +1,163 @@
+/*
+ * 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.test.mapred;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+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.mapred.AccumuloInputFormat;
+import org.apache.accumulo.core.client.mapred.AccumuloMultiTableInputFormat;
+import org.apache.accumulo.core.client.mapred.RangeInputSplit;
+import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Mapper;
+import org.apache.hadoop.mapred.OutputCollector;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.lib.NullOutputFormat;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.junit.Test;
+
+public class AccumuloMultiTableInputFormatIT extends AccumuloClusterHarness {
+
+  private static AssertionError e1 = null;
+  private static AssertionError e2 = null;
+
+  private static class MRTester extends Configured implements Tool {
+    private static class TestMapper implements Mapper<Key,Value,Key,Value> {
+      Key key = null;
+      int count = 0;
+
+      @Override
+      public void map(Key k, Value v, OutputCollector<Key,Value> output, Reporter reporter) throws IOException {
+        try {
+          String tableName = ((RangeInputSplit) reporter.getInputSplit()).getTableName();
+          if (key != null)
+            assertEquals(key.getRow().toString(), new String(v.get()));
+          assertEquals(new Text(String.format("%s_%09x", tableName, count + 1)), k.getRow());
+          assertEquals(String.format("%s_%09x", tableName, count), new String(v.get()));
+        } catch (AssertionError e) {
+          e1 = e;
+        }
+        key = new Key(k);
+        count++;
+      }
+
+      @Override
+      public void configure(JobConf job) {}
+
+      @Override
+      public void close() throws IOException {
+        try {
+          assertEquals(100, count);
+        } catch (AssertionError e) {
+          e2 = e;
+        }
+      }
+
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+      if (args.length != 2) {
+        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <table1> <table2>");
+      }
+
+      String user = getAdminPrincipal();
+      AuthenticationToken pass = getAdminToken();
+      String table1 = args[0];
+      String table2 = args[1];
+
+      JobConf job = new JobConf(getConf());
+      job.setJarByClass(this.getClass());
+
+      job.setInputFormat(AccumuloInputFormat.class);
+
+      AccumuloMultiTableInputFormat.setConnectorInfo(job, user, pass);
+      AccumuloMultiTableInputFormat.setZooKeeperInstance(job, getCluster().getClientConfig());
+
+      InputTableConfig tableConfig1 = new InputTableConfig();
+      InputTableConfig tableConfig2 = new InputTableConfig();
+
+      Map<String,InputTableConfig> configMap = new HashMap<String,InputTableConfig>();
+      configMap.put(table1, tableConfig1);
+      configMap.put(table2, tableConfig2);
+
+      AccumuloMultiTableInputFormat.setInputTableConfigs(job, configMap);
+
+      job.setMapperClass(TestMapper.class);
+      job.setMapOutputKeyClass(Key.class);
+      job.setMapOutputValueClass(Value.class);
+      job.setOutputFormat(NullOutputFormat.class);
+
+      job.setNumReduceTasks(0);
+
+      return JobClient.runJob(job).isSuccessful() ? 0 : 1;
+    }
+
+    public static void main(String[] args) throws Exception {
+      Configuration conf = new Configuration();
+      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
+      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
+    }
+  }
+
+  @Test
+  public void testMap() throws Exception {
+    String[] tableNames = getUniqueNames(2);
+    String table1 = tableNames[0];
+    String table2 = tableNames[1];
+    Connector c = getConnector();
+    c.tableOperations().create(table1);
+    c.tableOperations().create(table2);
+    BatchWriter bw = c.createBatchWriter(table1, new BatchWriterConfig());
+    BatchWriter bw2 = c.createBatchWriter(table2, new BatchWriterConfig());
+    for (int i = 0; i < 100; i++) {
+      Mutation t1m = new Mutation(new Text(String.format("%s_%09x", table1, i + 1)));
+      t1m.put(new Text(), new Text(), new Value(String.format("%s_%09x", table1, i).getBytes()));
+      bw.addMutation(t1m);
+      Mutation t2m = new Mutation(new Text(String.format("%s_%09x", table2, i + 1)));
+      t2m.put(new Text(), new Text(), new Value(String.format("%s_%09x", table2, i).getBytes()));
+      bw2.addMutation(t2m);
+    }
+    bw.close();
+    bw2.close();
+
+    MRTester.main(new String[] {table1, table2});
+    assertNull(e1);
+    assertNull(e2);
+  }
+
+}


[14/14] accumulo git commit: ACCUMULO-3949 Formatting

Posted by ct...@apache.org.
ACCUMULO-3949 Formatting

Apply formatting changes from previous change


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

Branch: refs/heads/master
Commit: a9e6d7348839517d1bcc8bb4c35e775e126231cb
Parents: da8d8a8
Author: Christopher Tubbs <ct...@apache.org>
Authored: Thu Jul 30 17:42:35 2015 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Thu Jul 30 17:42:35 2015 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/tserver/tablet/Compactor.java    | 11 +++--------
 1 file changed, 3 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a9e6d734/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
index c1b8105..4c55db9 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Compactor.java
@@ -219,14 +219,9 @@ public class Compactor implements Callable<CompactionStats> {
       mfw = null; // set this to null so we do not try to close it again in finally if the close fails
       mfwTmp.close(); // if the close fails it will cause the compaction to fail
 
-      log.debug(String.format("Compaction %s %,d read | %,d written | %,6d entries/sec | %,6.3f secs | %,12d bytes | %9.3f byte/sec",
-          extent,
-          majCStats.getEntriesRead(),
-          majCStats.getEntriesWritten(),
-          (int) (majCStats.getEntriesRead() / ((t2 - t1) / 1000.0)),
-          (t2 - t1) / 1000.0,
-          mfwTmp.getLength(),
-          mfwTmp.getLength() / ((t2 - t1) / 1000.0)));
+      log.debug(String.format("Compaction %s %,d read | %,d written | %,6d entries/sec | %,6.3f secs | %,12d bytes | %9.3f byte/sec", extent,
+          majCStats.getEntriesRead(), majCStats.getEntriesWritten(), (int) (majCStats.getEntriesRead() / ((t2 - t1) / 1000.0)), (t2 - t1) / 1000.0,
+          mfwTmp.getLength(), mfwTmp.getLength() / ((t2 - t1) / 1000.0)));
 
       majCStats.setFileSize(mfwTmp.getLength());
       return majCStats;


[12/14] accumulo git commit: ACCUMULO-3920 Deprecate mock components

Posted by ct...@apache.org.
ACCUMULO-3920 Deprecate mock components

* Deprecate Accumulo Mock classes
* Add javadoc to mock package to add additional details
* Create DeprecationUtil to help migrate code which must continue to use mock
* Minimize mock usage throughout the tests wherever possible
  * Use EasyMock in a few cases where it was possible to satisfy the test
  * Apply @Rule, @Before, etc. to reduce number of calls to MockInstance
* Create replacement IteratorAdapter for use outside of mock


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

Branch: refs/heads/master
Commit: 2556df3af159e80d0a0d7f06eff9360047ff135d
Parents: 6503223
Author: Christopher Tubbs <ct...@apache.org>
Authored: Wed Jul 8 22:02:03 2015 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Thu Jul 30 16:33:03 2015 -0400

----------------------------------------------------------------------
 core/src/main/findbugs/exclude-filter.xml       |   1 +
 .../apache/accumulo/core/cli/ClientOpts.java    |   4 +-
 .../core/client/ClientSideIteratorScanner.java  |   2 +-
 .../core/client/mapred/AbstractInputFormat.java |  12 +-
 .../core/client/mapred/AccumuloInputFormat.java |   2 +-
 .../mapred/AccumuloMultiTableInputFormat.java   |   2 +-
 .../client/mapred/AccumuloOutputFormat.java     |   8 +-
 .../client/mapred/AccumuloRowInputFormat.java   |   2 +-
 .../client/mapreduce/AbstractInputFormat.java   |  12 +-
 .../client/mapreduce/AccumuloInputFormat.java   |   2 +-
 .../AccumuloMultiTableInputFormat.java          |   2 +-
 .../client/mapreduce/AccumuloOutputFormat.java  |   7 +-
 .../mapreduce/AccumuloRowInputFormat.java       |   2 +-
 .../core/client/mapreduce/RangeInputSplit.java  |  16 +-
 .../core/client/mapreduce/impl/SplitUtils.java  |   4 +-
 .../mapreduce/lib/impl/ConfiguratorBase.java    |  11 +-
 .../mapreduce/lib/impl/InputConfigurator.java   |   4 +-
 .../mapreduce/lib/util/ConfiguratorBase.java    |   6 +-
 .../core/client/mock/IteratorAdapter.java       |  36 +----
 .../accumulo/core/client/mock/MockAccumulo.java |  10 +-
 .../core/client/mock/MockBatchDeleter.java      |   3 +
 .../core/client/mock/MockBatchScanner.java      |   4 +
 .../core/client/mock/MockBatchWriter.java       |   4 +
 .../core/client/mock/MockConfiguration.java     |   4 +
 .../core/client/mock/MockConnector.java         |   4 +
 .../accumulo/core/client/mock/MockInstance.java |   3 +-
 .../client/mock/MockInstanceOperations.java     |   4 +
 .../client/mock/MockMultiTableBatchWriter.java  |   4 +
 .../core/client/mock/MockNamespace.java         |   4 +
 .../client/mock/MockNamespaceOperations.java    |   4 +
 .../accumulo/core/client/mock/MockScanner.java  |   4 +
 .../core/client/mock/MockScannerBase.java       |   4 +
 .../client/mock/MockSecurityOperations.java     |   4 +
 .../accumulo/core/client/mock/MockTable.java    |   4 +
 .../core/client/mock/MockTableOperations.java   |   4 +
 .../accumulo/core/client/mock/MockUser.java     |   4 +
 .../client/mock/impl/MockTabletLocator.java     |   4 +
 .../accumulo/core/client/mock/package-info.java |  25 ++++
 .../core/iterators/IteratorAdapter.java         |  56 +++++++
 .../core/replication/ReplicationTable.java      |   4 +-
 .../accumulo/core/util/DeprecationUtil.java     |  61 ++++++++
 .../core/client/ClientSideIteratorTest.java     |  21 ++-
 .../accumulo/core/client/admin/FindMaxTest.java |  18 ++-
 .../core/client/impl/ScannerImplTest.java       |  10 +-
 .../impl/TabletServerBatchReaderTest.java       |  14 +-
 .../mapred/AccumuloFileOutputFormatTest.java    |   4 +-
 .../client/mapred/AccumuloInputFormatTest.java  |  33 ++--
 .../AccumuloMultiTableInputFormatTest.java      |   4 +-
 .../client/mapred/AccumuloOutputFormatTest.java |   4 +-
 .../mapred/AccumuloRowInputFormatTest.java      |  21 ++-
 .../core/client/mapred/RangeInputSplitTest.java |   5 +-
 .../core/client/mapred/TokenFileTest.java       |  25 ++--
 .../mapreduce/AccumuloFileOutputFormatTest.java |   4 +-
 .../mapreduce/AccumuloInputFormatTest.java      |  44 +++---
 .../AccumuloMultiTableInputFormatTest.java      |   4 +-
 .../mapreduce/AccumuloOutputFormatTest.java     |   4 +-
 .../mapreduce/AccumuloRowInputFormatTest.java   |   4 +-
 .../client/mapreduce/RangeInputSplitTest.java   |   5 +-
 .../core/client/mapreduce/TokenFileTest.java    |   4 +-
 .../mapreduce/impl/BatchInputSplitTest.java     |   6 +-
 .../lib/impl/ConfiguratorBaseTest.java          |   7 +-
 .../core/client/mock/MockConnectorTest.java     |   1 +
 .../core/client/mock/MockNamespacesTest.java    | 150 +++++++++----------
 .../client/mock/MockTableOperationsTest.java    |  57 ++++---
 .../core/client/mock/TestBatchScanner821.java   |  12 ++
 .../core/client/mock/TransformIterator.java     |  30 ----
 .../user/IntersectingIteratorTest.java          |  38 ++---
 .../core/iterators/user/RegExFilterTest.java    |  25 +++-
 .../core/iterators/user/RowFilterTest.java      |  29 ++--
 .../user/TransformingIteratorTest.java          |  13 +-
 .../core/metadata/MetadataServicerTest.java     |  54 +++++--
 .../accumulo/core/security/CredentialsTest.java |  29 +++-
 .../examples/simple/dirlist/CountTest.java      |  65 ++++----
 .../simple/filedata/ChunkInputFormatTest.java   |  28 ++--
 .../simple/filedata/ChunkInputStreamTest.java   |  24 ++-
 .../minicluster/MiniAccumuloCluster.java        |   2 +-
 .../impl/MiniAccumuloClusterImpl.java           |   5 +-
 .../org/apache/accumulo/proxy/ProxyServer.java  |   4 +-
 .../accumulo/server/AccumuloServerContext.java  |   4 +-
 .../server/cli/ClientOnDefaultTable.java        |   4 +-
 .../server/cli/ClientOnRequiredTable.java       |   4 +-
 .../apache/accumulo/server/cli/ClientOpts.java  |   4 +-
 .../server/AccumuloServerContextTest.java       |   4 +-
 .../server/client/BulkImporterTest.java         |  10 +-
 .../master/balancer/TableLoadBalancerTest.java  |   4 +-
 .../problems/ProblemReportingIteratorTest.java  |  16 +-
 .../server/security/SystemCredentialsTest.java  |  19 ++-
 .../apache/accumulo/server/util/CloneTest.java  |  53 +++----
 .../server/util/ReplicationTableUtilTest.java   |   6 +-
 .../server/util/TabletIteratorTest.java         |  29 ++--
 .../CloseWriteAheadLogReferencesTest.java       |  30 ++--
 .../master/ReplicationOperationsImplTest.java   |   5 +-
 .../apache/accumulo/master/TestMergeState.java  |  11 +-
 .../replication/FinishedWorkUpdaterTest.java    |   4 +-
 .../RemoveCompleteReplicationRecordsTest.java   |   6 +-
 .../replication/SequentialWorkAssignerTest.java |  60 ++------
 .../master/replication/StatusMakerTest.java     |  28 ++--
 .../replication/UnorderedWorkAssignerTest.java  |  38 ++---
 .../master/replication/WorkMakerTest.java       |   6 +-
 .../tserver/LargestFirstMemoryManagerTest.java  |  14 +-
 .../java/org/apache/accumulo/shell/Shell.java   |   6 +-
 .../apache/accumulo/shell/mock/MockShell.java   |   6 +-
 .../accumulo/shell/ShellSetInstanceTest.java    |   9 +-
 .../shell/commands/FormatterCommandTest.java    |   7 +-
 .../accumulo/test/iterator/RegExTest.java       |  31 ++--
 105 files changed, 917 insertions(+), 665 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/findbugs/exclude-filter.xml
----------------------------------------------------------------------
diff --git a/core/src/main/findbugs/exclude-filter.xml b/core/src/main/findbugs/exclude-filter.xml
index aa854f0..cd211a6 100644
--- a/core/src/main/findbugs/exclude-filter.xml
+++ b/core/src/main/findbugs/exclude-filter.xml
@@ -53,6 +53,7 @@
       <Class name="org.apache.accumulo.core.util.AddressUtil" />
       <Class name="org.apache.accumulo.core.zookeeper.ZooUtil" />
       <Class name="org.apache.accumulo.core.security.VisibilityConstraint" />
+      <Class name="org.apache.accumulo.core.client.mock.IteratorAdapter" />
     </Or>
     <Or>
       <Bug code="NM" pattern="NM_SAME_SIMPLE_NAME_AS_SUPERCLASS" />

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java b/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
index 54e8b53..71ee6d1 100644
--- a/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
+++ b/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java
@@ -34,7 +34,6 @@ import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.Properties;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
@@ -45,6 +44,7 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.trace.Trace;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.core.volume.VolumeConfiguration;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.hadoop.conf.Configuration;
@@ -260,7 +260,7 @@ public class ClientOpts extends Help {
     if (cachedInstance != null)
       return cachedInstance;
     if (mock)
-      return cachedInstance = new MockInstance(instance);
+      return cachedInstance = DeprecationUtil.makeMockInstance(instance);
     return cachedInstance = new ZooKeeperInstance(this.getClientConfiguration());
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
index 10931f5..eb3c923 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ClientSideIteratorScanner.java
@@ -28,7 +28,6 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.impl.ScannerOptions;
-import org.apache.accumulo.core.client.mock.IteratorAdapter;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
@@ -37,6 +36,7 @@ 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.data.thrift.IterInfo;
+import org.apache.accumulo.core.iterators.IteratorAdapter;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
index 2a25dd5..d11639e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
@@ -57,7 +57,6 @@ import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
 import org.apache.accumulo.core.client.mapreduce.impl.SplitUtils;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.DelegationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
@@ -68,6 +67,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.InputFormat;
@@ -229,7 +229,9 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
    * @param instanceName
    *          the Accumulo instance name
    * @since 1.5.0
+   * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework
    */
+  @Deprecated
   public static void setMockInstance(JobConf job, String instanceName) {
     InputConfigurator.setMockInstance(CLASS, job, instanceName);
   }
@@ -242,7 +244,6 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
    * @return an Accumulo instance
    * @since 1.5.0
    * @see #setZooKeeperInstance(JobConf, ClientConfiguration)
-   * @see #setMockInstance(JobConf, String)
    */
   protected static Instance getInstance(JobConf job) {
     return InputConfigurator.getInstance(CLASS, job);
@@ -531,7 +532,7 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
         try {
           if (isOffline) {
             scanner = new OfflineScanner(instance, new Credentials(principal, token), baseSplit.getTableId(), authorizations);
-          } else if (instance instanceof MockInstance) {
+          } else if (DeprecationUtil.isMockInstance(instance)) {
             scanner = instance.getConnector(principal, token).createScanner(baseSplit.getTableName(), authorizations);
           } else {
             ClientConfiguration clientConf = getClientConfiguration(job);
@@ -632,7 +633,7 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
       Instance instance = getInstance(job);
       String tableId;
       // resolve table name to id once, and use id from this point forward
-      if (instance instanceof MockInstance) {
+      if (DeprecationUtil.isMockInstance(instance)) {
         tableId = "";
       } else {
         try {
@@ -681,7 +682,7 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
           ClientContext context = new ClientContext(getInstance(job), new Credentials(getPrincipal(job), getAuthenticationToken(job)),
               getClientConfiguration(job));
           while (!tl.binRanges(context, ranges, binnedRanges).isEmpty()) {
-            if (!(instance instanceof MockInstance)) {
+            if (!DeprecationUtil.isMockInstance(instance)) {
               if (!Tables.exists(instance, tableId))
                 throw new TableDeletedException(tableId);
               if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
@@ -763,4 +764,5 @@ public abstract class AbstractInputFormat<K,V> implements InputFormat<K,V> {
 
     return splits.toArray(new InputSplit[splits.size()]);
   }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
index 856a11a..5f00ec3 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloInputFormat.java
@@ -42,7 +42,7 @@ import org.apache.log4j.Level;
  * <li>{@link AccumuloInputFormat#setConnectorInfo(JobConf, String, AuthenticationToken)}
  * <li>{@link AccumuloInputFormat#setConnectorInfo(JobConf, String, String)}
  * <li>{@link AccumuloInputFormat#setScanAuthorizations(JobConf, Authorizations)}
- * <li>{@link AccumuloInputFormat#setZooKeeperInstance(JobConf, ClientConfiguration)} OR {@link AccumuloInputFormat#setMockInstance(JobConf, String)}
+ * <li>{@link AccumuloInputFormat#setZooKeeperInstance(JobConf, ClientConfiguration)}
  * </ul>
  *
  * Other static methods are optional.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
index 00a79f2..3a2e3fa 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloMultiTableInputFormat.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.mapred.Reporter;
  * <li>{@link AccumuloInputFormat#setConnectorInfo(JobConf, String, org.apache.accumulo.core.client.security.tokens.AuthenticationToken)}
  * <li>{@link AccumuloInputFormat#setConnectorInfo(JobConf, String, String)}
  * <li>{@link AccumuloInputFormat#setScanAuthorizations(JobConf, org.apache.accumulo.core.security.Authorizations)}
- * <li>{@link AccumuloInputFormat#setZooKeeperInstance(JobConf, ClientConfiguration)} OR {@link AccumuloInputFormat#setMockInstance(JobConf, String)}
+ * <li>{@link AccumuloInputFormat#setZooKeeperInstance(JobConf, ClientConfiguration)}
  * <li>{@link AccumuloMultiTableInputFormat#setInputTableConfigs(org.apache.hadoop.mapred.JobConf, java.util.Map)}
  * </ul>
  *

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
index c194cf6..c24557a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
@@ -40,7 +40,6 @@ import org.apache.accumulo.core.client.impl.AuthenticationTokenIdentifier;
 import org.apache.accumulo.core.client.impl.DelegationTokenImpl;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.OutputConfigurator;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
@@ -71,7 +70,7 @@ import org.apache.log4j.Logger;
  * <ul>
  * <li>{@link AccumuloOutputFormat#setConnectorInfo(JobConf, String, AuthenticationToken)}
  * <li>{@link AccumuloOutputFormat#setConnectorInfo(JobConf, String, String)}
- * <li>{@link AccumuloOutputFormat#setZooKeeperInstance(JobConf, ClientConfiguration)} OR {@link AccumuloOutputFormat#setMockInstance(JobConf, String)}
+ * <li>{@link AccumuloOutputFormat#setZooKeeperInstance(JobConf, ClientConfiguration)}
  * </ul>
  *
  * Other static methods are optional.
@@ -239,14 +238,16 @@ public class AccumuloOutputFormat implements OutputFormat<Text,Mutation> {
   }
 
   /**
-   * Configures a {@link MockInstance} for this job.
+   * Configures a {@link org.apache.accumulo.core.client.mock.MockInstance} for this job.
    *
    * @param job
    *          the Hadoop job instance to be configured
    * @param instanceName
    *          the Accumulo instance name
    * @since 1.5.0
+   * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework
    */
+  @Deprecated
   public static void setMockInstance(JobConf job, String instanceName) {
     OutputConfigurator.setMockInstance(CLASS, job, instanceName);
   }
@@ -259,7 +260,6 @@ public class AccumuloOutputFormat implements OutputFormat<Text,Mutation> {
    * @return an Accumulo instance
    * @since 1.5.0
    * @see #setZooKeeperInstance(JobConf, ClientConfiguration)
-   * @see #setMockInstance(JobConf, String)
    */
   protected static Instance getInstance(JobConf job) {
     return OutputConfigurator.getInstance(CLASS, job);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java
index 6f257ff..067fde8 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.mapred.Reporter;
  * <li>{@link AccumuloRowInputFormat#setConnectorInfo(JobConf, String, AuthenticationToken)}
  * <li>{@link AccumuloRowInputFormat#setInputTableName(JobConf, String)}
  * <li>{@link AccumuloRowInputFormat#setScanAuthorizations(JobConf, Authorizations)}
- * <li>{@link AccumuloRowInputFormat#setZooKeeperInstance(JobConf, ClientConfiguration)} OR {@link AccumuloRowInputFormat#setMockInstance(JobConf, String)}
+ * <li>{@link AccumuloRowInputFormat#setZooKeeperInstance(JobConf, ClientConfiguration)}
  * </ul>
  *
  * Other static methods are optional.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
index 76f3f20..7db67c7 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java
@@ -56,7 +56,6 @@ import org.apache.accumulo.core.client.mapreduce.impl.BatchInputSplit;
 import org.apache.accumulo.core.client.mapreduce.impl.SplitUtils;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.DelegationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
@@ -67,6 +66,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
@@ -255,7 +255,9 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
    * @param instanceName
    *          the Accumulo instance name
    * @since 1.5.0
+   * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework
    */
+  @Deprecated
   public static void setMockInstance(Job job, String instanceName) {
     InputConfigurator.setMockInstance(CLASS, job.getConfiguration(), instanceName);
   }
@@ -268,7 +270,6 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
    * @return an Accumulo instance
    * @since 1.5.0
    * @see #setZooKeeperInstance(Job, ClientConfiguration)
-   * @see #setMockInstance(Job, String)
    */
   protected static Instance getInstance(JobContext context) {
     return InputConfigurator.getInstance(CLASS, context.getConfiguration());
@@ -561,7 +562,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
         try {
           if (isOffline) {
             scanner = new OfflineScanner(instance, new Credentials(principal, token), split.getTableId(), authorizations);
-          } else if (instance instanceof MockInstance) {
+          } else if (DeprecationUtil.isMockInstance(instance)) {
             scanner = instance.getConnector(principal, token).createScanner(split.getTableName(), authorizations);
           } else {
             ClientConfiguration clientConf = getClientConfiguration(attempt);
@@ -679,7 +680,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
       Instance instance = getInstance(context);
       String tableId;
       // resolve table name to id once, and use id from this point forward
-      if (instance instanceof MockInstance) {
+      if (DeprecationUtil.isMockInstance(instance)) {
         tableId = "";
       } else {
         try {
@@ -729,7 +730,7 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
           ClientContext clientContext = new ClientContext(getInstance(context), new Credentials(getPrincipal(context), getAuthenticationToken(context)),
               getClientConfiguration(context));
           while (!tl.binRanges(clientContext, ranges, binnedRanges).isEmpty()) {
-            if (!(instance instanceof MockInstance)) {
+            if (!DeprecationUtil.isMockInstance(instance)) {
               if (!Tables.exists(instance, tableId))
                 throw new TableDeletedException(tableId);
               if (Tables.getTableState(instance, tableId) == TableState.OFFLINE)
@@ -811,4 +812,5 @@ public abstract class AbstractInputFormat<K,V> extends InputFormat<K,V> {
     }
     return splits;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
index 33eccc1..837b3fe 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloInputFormat.java
@@ -41,7 +41,7 @@ import org.apache.log4j.Level;
  * <ul>
  * <li>{@link AccumuloInputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
  * <li>{@link AccumuloInputFormat#setScanAuthorizations(Job, Authorizations)}
- * <li>{@link AccumuloInputFormat#setZooKeeperInstance(Job, ClientConfiguration)} OR {@link AccumuloInputFormat#setMockInstance(Job, String)}
+ * <li>{@link AccumuloInputFormat#setZooKeeperInstance(Job, ClientConfiguration)}
  * </ul>
  *
  * Other static methods are optional.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
index 679256b..c7ea1f7 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloMultiTableInputFormat.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
  * <ul>
  * <li>{@link AccumuloMultiTableInputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
  * <li>{@link AccumuloMultiTableInputFormat#setScanAuthorizations(Job, Authorizations)}
- * <li>{@link AccumuloMultiTableInputFormat#setZooKeeperInstance(Job, ClientConfiguration)} OR {@link AccumuloInputFormat#setMockInstance(Job, String)}
+ * <li>{@link AccumuloMultiTableInputFormat#setZooKeeperInstance(Job, ClientConfiguration)}
  * <li>{@link AccumuloMultiTableInputFormat#setInputTableConfigs(Job, Map)}
  * </ul>
  *

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
index 4cb46a3..42bd243 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java
@@ -40,7 +40,6 @@ import org.apache.accumulo.core.client.impl.AuthenticationTokenIdentifier;
 import org.apache.accumulo.core.client.impl.DelegationTokenImpl;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.OutputConfigurator;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
@@ -72,7 +71,7 @@ import org.apache.log4j.Logger;
  * <ul>
  * <li>{@link AccumuloOutputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
  * <li>{@link AccumuloOutputFormat#setConnectorInfo(Job, String, String)}
- * <li>{@link AccumuloOutputFormat#setZooKeeperInstance(Job, ClientConfiguration)} OR {@link AccumuloOutputFormat#setMockInstance(Job, String)}
+ * <li>{@link AccumuloOutputFormat#setZooKeeperInstance(Job, ClientConfiguration)}
  * </ul>
  *
  * Other static methods are optional.
@@ -239,7 +238,7 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
   }
 
   /**
-   * Configures a {@link MockInstance} for this job.
+   * Configures a {@link org.apache.accumulo.core.client.mock.MockInstance} for this job.
    *
    * @param job
    *          the Hadoop job instance to be configured
@@ -247,6 +246,7 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
    *          the Accumulo instance name
    * @since 1.5.0
    */
+  @Deprecated
   public static void setMockInstance(Job job, String instanceName) {
     OutputConfigurator.setMockInstance(CLASS, job.getConfiguration(), instanceName);
   }
@@ -259,7 +259,6 @@ public class AccumuloOutputFormat extends OutputFormat<Text,Mutation> {
    * @return an Accumulo instance
    * @since 1.5.0
    * @see #setZooKeeperInstance(Job, ClientConfiguration)
-   * @see #setMockInstance(Job, String)
    */
   protected static Instance getInstance(JobContext context) {
     return OutputConfigurator.getInstance(CLASS, context.getConfiguration());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
index 77081bf..7dcb35b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
  * <li>{@link AccumuloRowInputFormat#setConnectorInfo(Job, String, AuthenticationToken)}
  * <li>{@link AccumuloRowInputFormat#setInputTableName(Job, String)}
  * <li>{@link AccumuloRowInputFormat#setScanAuthorizations(Job, Authorizations)}
- * <li>{@link AccumuloRowInputFormat#setZooKeeperInstance(Job, ClientConfiguration)} OR {@link AccumuloRowInputFormat#setMockInstance(Job, String)}
+ * <li>{@link AccumuloRowInputFormat#setZooKeeperInstance(Job, ClientConfiguration)}
  * </ul>
  *
  * Other static methods are optional.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
index e337977..f3e17c6 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java
@@ -16,6 +16,8 @@
  */
 package org.apache.accumulo.core.client.mapreduce;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -33,7 +35,6 @@ import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.mapreduce.impl.SplitUtils;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.ConfiguratorBase.TokenSource;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.InputConfigurator;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
 import org.apache.accumulo.core.data.ByteSequence;
@@ -42,14 +43,13 @@ import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.Base64;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.log4j.Level;
 
-import static java.nio.charset.StandardCharsets.UTF_8;
-
 /**
  * The Class RangeInputSplit. Encapsulates an Accumulo range for use in Map Reduce jobs.
  */
@@ -354,7 +354,7 @@ public class RangeInputSplit extends InputSplit implements Writable {
     }
 
     if (isMockInstance()) {
-      return new MockInstance(getInstanceName());
+      return DeprecationUtil.makeMockInstance(getInstanceName());
     }
 
     if (null == zooKeepers) {
@@ -414,10 +414,18 @@ public class RangeInputSplit extends InputSplit implements Writable {
     this.locations = Arrays.copyOf(locations, locations.length);
   }
 
+  /**
+   * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework
+   */
+  @Deprecated
   public Boolean isMockInstance() {
     return mockInstance;
   }
 
+  /**
+   * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework
+   */
+  @Deprecated
   public void setMockInstance(Boolean mockInstance) {
     this.mockInstance = mockInstance;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/SplitUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/SplitUtils.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/SplitUtils.java
index d19b499..68268fc 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/SplitUtils.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/SplitUtils.java
@@ -23,11 +23,11 @@ import java.math.BigInteger;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
 import org.apache.accumulo.core.client.mapreduce.RangeInputSplit;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Level;
 
@@ -41,7 +41,7 @@ public class SplitUtils {
       Authorizations auths, Level logLevel) {
     split.setInstanceName(instance.getInstanceName());
     split.setZooKeepers(instance.getZooKeepers());
-    split.setMockInstance(instance instanceof MockInstance);
+    DeprecationUtil.setMockInstance(split, DeprecationUtil.isMockInstance(instance));
 
     split.setPrincipal(principal);
     split.setToken(token);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java
index 5364e4a..f2a0c06 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/ConfiguratorBase.java
@@ -35,10 +35,10 @@ import org.apache.accumulo.core.client.impl.AuthenticationTokenIdentifier;
 import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.core.client.impl.DelegationTokenImpl;
 import org.apache.accumulo.core.client.mapreduce.impl.DelegationTokenStub;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
 import org.apache.accumulo.core.util.Base64;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -81,7 +81,7 @@ public class ConfiguratorBase {
   }
 
   /**
-   * Configuration keys for {@link Instance}, {@link ZooKeeperInstance}, and {@link MockInstance}.
+   * Configuration keys for available {@link Instance} types.
    *
    * @since 1.6.0
    */
@@ -321,7 +321,7 @@ public class ConfiguratorBase {
   }
 
   /**
-   * Configures a {@link MockInstance} for this job.
+   * Configures a {@link org.apache.accumulo.core.client.mock.MockInstance} for this job.
    *
    * @param implementingClass
    *          the class whose name will be used as a prefix for the property configuration key
@@ -330,7 +330,9 @@ public class ConfiguratorBase {
    * @param instanceName
    *          the Accumulo instance name
    * @since 1.6.0
+   * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework
    */
+  @Deprecated
   public static void setMockInstance(Class<?> implementingClass, Configuration conf, String instanceName) {
     String key = enumToConfKey(implementingClass, InstanceOpts.TYPE);
     if (!conf.get(key, "").isEmpty())
@@ -351,12 +353,11 @@ public class ConfiguratorBase {
    * @return an Accumulo instance
    * @since 1.6.0
    * @see #setZooKeeperInstance(Class, Configuration, ClientConfiguration)
-   * @see #setMockInstance(Class, Configuration, String)
    */
   public static Instance getInstance(Class<?> implementingClass, Configuration conf) {
     String instanceType = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE), "");
     if ("MockInstance".equals(instanceType))
-      return new MockInstance(conf.get(enumToConfKey(implementingClass, InstanceOpts.NAME)));
+      return DeprecationUtil.makeMockInstance(conf.get(enumToConfKey(implementingClass, InstanceOpts.NAME)));
     else if ("ZooKeeperInstance".equals(instanceType)) {
       return new ZooKeeperInstance(getClientConfiguration(implementingClass, conf));
     } else if (instanceType.isEmpty())

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
index 0e640b4..efda7d9 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java
@@ -52,7 +52,6 @@ import org.apache.accumulo.core.client.impl.DelegationTokenImpl;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
-import org.apache.accumulo.core.client.mock.impl.MockTabletLocator;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.PartialKey;
@@ -66,6 +65,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.util.Base64;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.hadoop.conf.Configuration;
@@ -645,7 +645,7 @@ public class InputConfigurator extends ConfiguratorBase {
   public static TabletLocator getTabletLocator(Class<?> implementingClass, Configuration conf, String tableId) throws TableNotFoundException {
     String instanceType = conf.get(enumToConfKey(implementingClass, InstanceOpts.TYPE));
     if ("MockInstance".equals(instanceType))
-      return new MockTabletLocator();
+      return DeprecationUtil.makeMockLocator();
     Instance instance = getInstance(implementingClass, conf);
     ClientConfiguration clientConf = getClientConfiguration(implementingClass, conf);
     ClientContext context = new ClientContext(instance,

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
index 20fbbea..6914071 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/ConfiguratorBase.java
@@ -20,7 +20,6 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken.AuthenticationTokenSerializer;
 import org.apache.hadoop.conf.Configuration;
@@ -46,7 +45,7 @@ public class ConfiguratorBase {
   }
 
   /**
-   * Configuration keys for {@link Instance}, {@link ZooKeeperInstance}, and {@link MockInstance}.
+   * Configuration keys for {@link Instance}, {@link ZooKeeperInstance}, and {@link org.apache.accumulo.core.client.mock.MockInstance}.
    *
    * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
    * @since 1.5.0
@@ -206,7 +205,7 @@ public class ConfiguratorBase {
   }
 
   /**
-   * Configures a {@link MockInstance} for this job.
+   * Configures a {@link org.apache.accumulo.core.client.mock.MockInstance} for this job.
    *
    * @param implementingClass
    *          the class whose name will be used as a prefix for the property configuration key
@@ -233,7 +232,6 @@ public class ConfiguratorBase {
    * @deprecated since 1.6.0; Configure your job with the appropriate InputFormat or OutputFormat.
    * @since 1.5.0
    * @see #setZooKeeperInstance(Class, Configuration, String, String)
-   * @see #setMockInstance(Class, Configuration, String)
    */
   @Deprecated
   public static Instance getInstance(Class<?> implementingClass, Configuration conf) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/IteratorAdapter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/IteratorAdapter.java b/core/src/main/java/org/apache/accumulo/core/client/mock/IteratorAdapter.java
index d4d4004..d88dac9 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/IteratorAdapter.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/IteratorAdapter.java
@@ -16,42 +16,18 @@
  */
 package org.apache.accumulo.core.client.mock;
 
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.Map.Entry;
-import java.util.NoSuchElementException;
-
 import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyValue;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 
-public class IteratorAdapter implements Iterator<Entry<Key,Value>> {
-
-  SortedKeyValueIterator<Key,Value> inner;
+/**
+ * @deprecated since 1.8.0; use {@link org.apache.accumulo.core.iterators.IteratorAdapter} instead.
+ */
+@Deprecated
+public class IteratorAdapter extends org.apache.accumulo.core.iterators.IteratorAdapter {
 
   public IteratorAdapter(SortedKeyValueIterator<Key,Value> inner) {
-    this.inner = inner;
-  }
-
-  @Override
-  public boolean hasNext() {
-    return inner.hasTop();
-  }
-
-  @Override
-  public Entry<Key,Value> next() {
-    try {
-      Entry<Key,Value> result = new KeyValue(new Key(inner.getTopKey()), new Value(inner.getTopValue()).get());
-      inner.next();
-      return result;
-    } catch (IOException ex) {
-      throw new NoSuchElementException();
-    }
+    super(inner);
   }
 
-  @Override
-  public void remove() {
-    throw new UnsupportedOperationException();
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
index e1ca768..18dac55 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java
@@ -40,6 +40,10 @@ import org.apache.accumulo.core.security.TablePermission;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.Text;
 
+/**
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
+ */
+@Deprecated
 public class MockAccumulo {
   final Map<String,MockTable> tables = new HashMap<String,MockTable>();
   final Map<String,MockNamespace> namespaces = new HashMap<String,MockNamespace>();
@@ -48,11 +52,8 @@ public class MockAccumulo {
   final FileSystem fs;
   final AtomicInteger tableIdCounter = new AtomicInteger(0);
 
+  @Deprecated
   MockAccumulo(FileSystem fs) {
-    this.fs = fs;
-  }
-
-  {
     MockUser root = new MockUser("root", new PasswordToken(new byte[0]), Authorizations.EMPTY);
     root.permissions.add(SystemPermission.SYSTEM);
     users.put(root.name, root);
@@ -61,6 +62,7 @@ public class MockAccumulo {
     createTable("root", RootTable.NAME, true, TimeType.LOGICAL);
     createTable("root", MetadataTable.NAME, true, TimeType.LOGICAL);
     createTable("root", ReplicationTable.NAME, true, TimeType.LOGICAL);
+    this.fs = fs;
   }
 
   public FileSystem getFileSystem() {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchDeleter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchDeleter.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchDeleter.java
index bb9f2c8..bacd844 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchDeleter.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchDeleter.java
@@ -37,7 +37,10 @@ import org.apache.accumulo.core.security.ColumnVisibility;
  * </ol>
  *
  * Otherwise, it behaves as expected.
+ *
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
  */
+@Deprecated
 public class MockBatchDeleter extends MockBatchScanner implements BatchDeleter {
 
   private final MockAccumulo acc;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchScanner.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchScanner.java
index 4034271..0622f33 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchScanner.java
@@ -32,6 +32,10 @@ import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.commons.collections.iterators.IteratorChain;
 
+/**
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
+ */
+@Deprecated
 public class MockBatchScanner extends MockScannerBase implements BatchScanner {
 
   List<Range> ranges = null;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchWriter.java
index 163587f..53a0ddc 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchWriter.java
@@ -22,6 +22,10 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.data.Mutation;
 
+/**
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
+ */
+@Deprecated
 public class MockBatchWriter implements BatchWriter {
 
   final String tablename;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConfiguration.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConfiguration.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConfiguration.java
index 410105b..244d6f8 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConfiguration.java
@@ -24,6 +24,10 @@ import org.apache.accumulo.core.conf.Property;
 
 import com.google.common.base.Predicate;
 
+/**
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
+ */
+@Deprecated
 class MockConfiguration extends AccumuloConfiguration {
   Map<String,String> map;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
index d348400..9b5601b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockConnector.java
@@ -40,6 +40,10 @@ import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.security.tokens.NullToken;
 import org.apache.accumulo.core.security.Authorizations;
 
+/**
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
+ */
+@Deprecated
 public class MockConnector extends Connector {
 
   String username;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
index 57cd5ee..a37706a 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java
@@ -49,8 +49,9 @@ import org.apache.hadoop.io.Text;
  * An alternative to Mock Accumulo called MiniAccumuloCluster was introduced in Accumulo 1.5. MiniAccumuloCluster spins up actual Accumulo server processes, can
  * be used for unit testing, and its behavior should match Accumulo. The drawback of MiniAccumuloCluster is that it starts more slowly than Mock Accumulo.
  *
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
  */
-
+@Deprecated
 public class MockInstance implements Instance {
 
   static final String genericAddress = "localhost:1234";

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java
index c1acc04..b7e8e68 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java
@@ -29,6 +29,10 @@ import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
+ */
+@Deprecated
 class MockInstanceOperations implements InstanceOperations {
   private static final Logger log = LoggerFactory.getLogger(MockInstanceOperations.class);
   MockAccumulo acu;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/MockMultiTableBatchWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockMultiTableBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockMultiTableBatchWriter.java
index 9cc3dfb..cfacccb 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockMultiTableBatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockMultiTableBatchWriter.java
@@ -26,6 +26,10 @@ import org.apache.accumulo.core.client.MultiTableBatchWriter;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 
+/**
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
+ */
+@Deprecated
 public class MockMultiTableBatchWriter implements MultiTableBatchWriter {
   MockAccumulo acu = null;
   Map<String,MockBatchWriter> bws = null;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java
index 955564f..da118c3 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java
@@ -27,6 +27,10 @@ import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.security.NamespacePermission;
 
+/**
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
+ */
+@Deprecated
 public class MockNamespace {
 
   final HashMap<String,String> settings;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
index 004124d..0c657a9 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java
@@ -34,6 +34,10 @@ import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
+ */
+@Deprecated
 class MockNamespaceOperations extends NamespaceOperationsHelper {
 
   private static final Logger log = LoggerFactory.getLogger(MockNamespaceOperations.class);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/MockScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockScanner.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockScanner.java
index a9b6fd5..1e36964 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockScanner.java
@@ -30,6 +30,10 @@ import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.accumulo.core.security.Authorizations;
 
+/**
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
+ */
+@Deprecated
 public class MockScanner extends MockScannerBase implements Scanner {
 
   int batchSize = 0;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java
index 3c746e1..f81e9dd 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java
@@ -43,6 +43,10 @@ import org.apache.accumulo.core.iterators.system.VisibilityFilter;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.commons.lang.NotImplementedException;
 
+/**
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
+ */
+@Deprecated
 public class MockScannerBase extends ScannerOptions implements ScannerBase {
 
   protected final MockTable table;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
index cc51a47..bf4b46e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockSecurityOperations.java
@@ -32,6 +32,10 @@ import org.apache.accumulo.core.security.NamespacePermission;
 import org.apache.accumulo.core.security.SystemPermission;
 import org.apache.accumulo.core.security.TablePermission;
 
+/**
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
+ */
+@Deprecated
 class MockSecurityOperations implements SecurityOperations {
 
   final private MockAccumulo acu;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
index 6f66c60..fa0fbcd 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java
@@ -40,6 +40,10 @@ import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.hadoop.io.Text;
 
+/**
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
+ */
+@Deprecated
 public class MockTable {
 
   static class MockMemKey extends Key {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
index e998722..0712f22 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java
@@ -66,6 +66,10 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
 
+/**
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
+ */
+@Deprecated
 class MockTableOperations extends TableOperationsHelper {
   private static final Logger log = LoggerFactory.getLogger(MockTableOperations.class);
   private static final byte[] ZERO = {0};

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/MockUser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockUser.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockUser.java
index efc896e..e32edad 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockUser.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockUser.java
@@ -22,6 +22,10 @@ import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.SystemPermission;
 
+/**
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
+ */
+@Deprecated
 public class MockUser {
   final EnumSet<SystemPermission> permissions;
   final String name;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/impl/MockTabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/impl/MockTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/mock/impl/MockTabletLocator.java
index 1c0c6a9..9936709 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mock/impl/MockTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/impl/MockTabletLocator.java
@@ -32,6 +32,10 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.hadoop.io.Text;
 
+/**
+ * @deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
+ */
+@Deprecated
 public class MockTabletLocator extends TabletLocator {
   public MockTabletLocator() {}
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/client/mock/package-info.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/package-info.java b/core/src/main/java/org/apache/accumulo/core/client/mock/package-info.java
new file mode 100644
index 0000000..cdd5593
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/mock/package-info.java
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+/**
+ * Mock framework for Accumulo
+ *
+ * <p>
+ * Deprecated since 1.8.0; use MiniAccumuloCluster or a standard mock framework instead.
+ */
+@Deprecated
+package org.apache.accumulo.core.client.mock;
+

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorAdapter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorAdapter.java b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorAdapter.java
new file mode 100644
index 0000000..2d8af8f
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorAdapter.java
@@ -0,0 +1,56 @@
+/*
+ * 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 java.io.IOException;
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.NoSuchElementException;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.KeyValue;
+import org.apache.accumulo.core.data.Value;
+
+public class IteratorAdapter implements Iterator<Entry<Key,Value>> {
+
+  SortedKeyValueIterator<Key,Value> inner;
+
+  public IteratorAdapter(SortedKeyValueIterator<Key,Value> inner) {
+    this.inner = inner;
+  }
+
+  @Override
+  public boolean hasNext() {
+    return inner.hasTop();
+  }
+
+  @Override
+  public Entry<Key,Value> next() {
+    try {
+      Entry<Key,Value> result = new KeyValue(new Key(inner.getTopKey()), new Value(inner.getTopValue()).get());
+      inner.next();
+      return result;
+    } catch (IOException ex) {
+      throw new NoSuchElementException();
+    }
+  }
+
+  @Override
+  public void remove() {
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java
index 4b61b53..7076757 100644
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationTable.java
@@ -31,11 +31,11 @@ import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.master.state.tables.TableState;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.DeprecationUtil;
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -87,7 +87,7 @@ public class ReplicationTable {
   }
 
   public static boolean isOnline(Connector conn) {
-    return conn.getInstance() instanceof MockInstance || TableState.ONLINE == Tables.getTableState(conn.getInstance(), ID);
+    return DeprecationUtil.isMockInstance(conn.getInstance()) || TableState.ONLINE == Tables.getTableState(conn.getInstance(), ID);
   }
 
   public static void setOnline(Connector conn) throws AccumuloSecurityException, AccumuloException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/main/java/org/apache/accumulo/core/util/DeprecationUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/DeprecationUtil.java b/core/src/main/java/org/apache/accumulo/core/util/DeprecationUtil.java
new file mode 100644
index 0000000..cd798bb
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/DeprecationUtil.java
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.TabletLocator;
+import org.apache.accumulo.core.client.mapreduce.RangeInputSplit;
+
+/**
+ * A utility class for managing deprecated items. This avoids scattering private helper methods all over the code with warnings suppression.
+ *
+ * <p>
+ * This class will never be public API and methods will be removed as soon as they are no longer needed. No methods in this class will, themselves, be
+ * deprecated, because that would propagate the deprecation warning we are trying to avoid.
+ *
+ * <p>
+ * This class should not be used as a substitute for deprecated classes. It should <b>only</b> be used for implementation code which must remain to support the
+ * deprecated features, and <b>only</b> until that feature is removed.
+ */
+public class DeprecationUtil {
+
+  @SuppressWarnings("deprecation")
+  public static boolean isMockInstance(Instance instance) {
+    return instance instanceof org.apache.accumulo.core.client.mock.MockInstance;
+  }
+
+  @SuppressWarnings("deprecation")
+  public static Instance makeMockInstance(String instance) {
+    return new org.apache.accumulo.core.client.mock.MockInstance(instance);
+  }
+
+  @SuppressWarnings("deprecation")
+  public static void setMockInstance(RangeInputSplit split, boolean isMockInstance) {
+    split.setMockInstance(isMockInstance);
+  }
+
+  @SuppressWarnings("deprecation")
+  public static boolean isMockInstanceSet(RangeInputSplit split) {
+    return split.isMockInstance();
+  }
+
+  @SuppressWarnings("deprecation")
+  public static TabletLocator makeMockLocator() {
+    return new org.apache.accumulo.core.client.mock.impl.MockTabletLocator();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java
index 60f668b..e0ce90e 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map.Entry;
 
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -34,7 +33,10 @@ import org.apache.accumulo.core.iterators.user.IntersectingIterator;
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TestName;
 
 public class ClientSideIteratorTest {
   List<Key> resultSet1;
@@ -54,7 +56,7 @@ public class ClientSideIteratorTest {
     resultSet3.add(new Key("part2", "", "DOC2"));
   }
 
-  public void checkResults(final Iterable<Entry<Key,Value>> scanner, final List<Key> results, final PartialKey pk) {
+  private void checkResults(final Iterable<Entry<Key,Value>> scanner, final List<Key> results, final PartialKey pk) {
     int i = 0;
     for (Entry<Key,Value> entry : scanner) {
       assertTrue(entry.getKey().equals(results.get(i++), pk));
@@ -62,10 +64,19 @@ public class ClientSideIteratorTest {
     assertEquals(i, results.size());
   }
 
+  @Rule
+  public TestName test = new TestName();
+
+  private Connector conn;
+
+  @Before
+  public void setupInstance() throws Exception {
+    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
+    conn = inst.getConnector("root", new PasswordToken(""));
+  }
+
   @Test
   public void testIntersect() throws Exception {
-    Instance instance = new MockInstance("local");
-    Connector conn = instance.getConnector("root", new PasswordToken(""));
     conn.tableOperations().create("intersect");
     BatchWriter bw = conn.createBatchWriter("intersect", new BatchWriterConfig());
     Mutation m = new Mutation("part1");
@@ -94,8 +105,6 @@ public class ClientSideIteratorTest {
 
   @Test
   public void testVersioning() throws Exception {
-    final Instance instance = new MockInstance("local");
-    final Connector conn = instance.getConnector("root", new PasswordToken(""));
     conn.tableOperations().create("table");
     conn.tableOperations().removeProperty("table", "table.iterator.scan.vers");
     conn.tableOperations().removeProperty("table", "table.iterator.majc.vers");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/admin/FindMaxTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/admin/FindMaxTest.java b/core/src/test/java/org/apache/accumulo/core/client/admin/FindMaxTest.java
index 2dc6ba5..78f7e75 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/admin/FindMaxTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/admin/FindMaxTest.java
@@ -16,24 +16,28 @@
  */
 package org.apache.accumulo.core.client.admin;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
 import java.util.ArrayList;
 import java.util.Map.Entry;
 
-import junit.framework.TestCase;
-
 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.Scanner;
-import org.apache.accumulo.core.client.mock.MockInstance;
 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.Value;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
 
-public class FindMaxTest extends TestCase {
+public class FindMaxTest {
 
   private static Mutation nm(byte[] row) {
     Mutation m = new Mutation(new Text(row));
@@ -47,8 +51,12 @@ public class FindMaxTest extends TestCase {
     return m;
   }
 
+  @Rule
+  public TestName test = new TestName();
+
+  @Test
   public void test1() throws Exception {
-    MockInstance mi = new MockInstance();
+    Instance mi = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
 
     Connector conn = mi.getConnector("root", new PasswordToken(""));
     conn.tableOperations().create("foo");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerImplTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerImplTest.java
index 95b0903..eedc61d 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/ScannerImplTest.java
@@ -18,11 +18,9 @@ package org.apache.accumulo.core.client.impl;
 
 import static org.junit.Assert.assertEquals;
 
-import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.mock.MockInstance;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.security.Authorizations;
+import org.easymock.EasyMock;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -32,13 +30,11 @@ import org.junit.Test;
  */
 public class ScannerImplTest {
 
-  MockInstance instance;
-  ClientContext context;
+  private ClientContext context;
 
   @Before
   public void setup() {
-    instance = new MockInstance();
-    context = new ClientContext(instance, new Credentials("root", new PasswordToken("")), new ClientConfiguration());
+    context = EasyMock.createMock(ClientContext.class);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderTest.java
index 23c223e..b31050a 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderTest.java
@@ -16,25 +16,21 @@
  */
 package org.apache.accumulo.core.client.impl;
 
+import static org.junit.Assert.assertEquals;
+
 import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.ClientConfiguration;
-import org.apache.accumulo.core.client.mock.MockInstance;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.security.Authorizations;
+import org.easymock.EasyMock;
 import org.junit.Before;
 import org.junit.Test;
 
-import static org.junit.Assert.assertEquals;
-
 public class TabletServerBatchReaderTest {
 
-  MockInstance instance;
-  ClientContext context;
+  private ClientContext context;
 
   @Before
   public void setup() {
-    instance = new MockInstance();
-    context = new ClientContext(instance, new Credentials("root", new PasswordToken("")), new ClientConfiguration());
+    context = EasyMock.createMock(ClientContext.class);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2556df3a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
index c4a4a29..9bb85f4 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapred/AccumuloFileOutputFormatTest.java
@@ -29,9 +29,9 @@ import java.io.IOException;
 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.mapreduce.lib.impl.ConfiguratorBase;
 import org.apache.accumulo.core.client.mapreduce.lib.impl.FileOutputConfigurator;
-import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -73,7 +73,7 @@ public class AccumuloFileOutputFormatTest {
 
   @BeforeClass
   public static void setup() throws Exception {
-    MockInstance mockInstance = new MockInstance(INSTANCE_NAME);
+    Instance mockInstance = new org.apache.accumulo.core.client.mock.MockInstance(INSTANCE_NAME);
     Connector c = mockInstance.getConnector("root", new PasswordToken(""));
     c.tableOperations().create(EMPTY_TABLE);
     c.tableOperations().create(TEST_TABLE);


[06/14] accumulo git commit: ACCUMULO-3920 Convert more tests from mock

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
index c02ca66..94ef555 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormatTest.java
@@ -17,120 +17,17 @@
 package org.apache.accumulo.core.client.mapreduce;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
 
-import java.io.File;
 import java.io.IOException;
-import java.util.Iterator;
-import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.Scanner;
-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.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
 import org.junit.Test;
 
-/**
- *
- */
 public class AccumuloOutputFormatTest {
-  private static AssertionError e1 = null;
-  private static final String PREFIX = AccumuloOutputFormatTest.class.getSimpleName();
-  private static final String INSTANCE_NAME = PREFIX + "_mapreduce_instance";
-  private static final String TEST_TABLE_1 = PREFIX + "_mapreduce_table_1";
-  private static final String TEST_TABLE_2 = PREFIX + "_mapreduce_table_2";
-
-  private static class MRTester extends Configured implements Tool {
-    private static class TestMapper extends Mapper<Key,Value,Text,Mutation> {
-      Key key = null;
-      int count = 0;
-
-      @Override
-      protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
-        try {
-          if (key != null)
-            assertEquals(key.getRow().toString(), new String(v.get()));
-          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
-          assertEquals(new String(v.get()), String.format("%09x", count));
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        key = new Key(k);
-        count++;
-      }
-
-      @Override
-      protected void cleanup(Context context) throws IOException, InterruptedException {
-        Mutation m = new Mutation("total");
-        m.put("", "", Integer.toString(count));
-        context.write(new Text(), m);
-      }
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 4) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <inputtable> <outputtable>");
-      }
-
-      String user = args[0];
-      String pass = args[1];
-      String table1 = args[2];
-      String table2 = args[3];
-
-      Job job = Job.getInstance(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormatClass(AccumuloInputFormat.class);
-
-      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
-      AccumuloInputFormat.setInputTableName(job, table1);
-      AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormatClass(AccumuloOutputFormat.class);
-      job.setOutputKeyClass(Text.class);
-      job.setOutputValueClass(Mutation.class);
-
-      AccumuloOutputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
-      AccumuloOutputFormat.setCreateTables(job, false);
-      AccumuloOutputFormat.setDefaultTableName(job, table2);
-      AccumuloOutputFormat.setMockInstance(job, INSTANCE_NAME);
-
-      job.setNumReduceTasks(0);
-
-      job.waitForCompletion(true);
-
-      return job.isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
 
   @Test
   public void testBWSettings() throws IOException {
@@ -172,28 +69,4 @@ public class AccumuloOutputFormatTest {
     myAOF.checkOutputSpecs(job);
   }
 
-  @Test
-  public void testMR() throws Exception {
-    Instance mockInstance = new org.apache.accumulo.core.client.mock.MockInstance(INSTANCE_NAME);
-    Connector c = mockInstance.getConnector("root", new PasswordToken(""));
-    c.tableOperations().create(TEST_TABLE_1);
-    c.tableOperations().create(TEST_TABLE_2);
-    BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
-    for (int i = 0; i < 100; i++) {
-      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
-      bw.addMutation(m);
-    }
-    bw.close();
-
-    MRTester.main(new String[] {"root", "", TEST_TABLE_1, TEST_TABLE_2});
-    assertNull(e1);
-
-    Scanner scanner = c.createScanner(TEST_TABLE_2, new Authorizations());
-    Iterator<Entry<Key,Value>> iter = scanner.iterator();
-    assertTrue(iter.hasNext());
-    Entry<Key,Value> entry = iter.next();
-    assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
-    assertFalse(iter.hasNext());
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
deleted file mode 100644
index 8df9d0f..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormatTest.java
+++ /dev/null
@@ -1,204 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map.Entry;
-
-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.MutationsRejectedException;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyValue;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.util.PeekingIterator;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.Test;
-
-public class AccumuloRowInputFormatTest {
-  private static final String PREFIX = AccumuloRowInputFormatTest.class.getSimpleName();
-  private static final String INSTANCE_NAME = PREFIX + "_mapreduce_instance";
-  private static final String TEST_TABLE_1 = PREFIX + "_mapreduce_table_1";
-
-  private static final String ROW1 = "row1";
-  private static final String ROW2 = "row2";
-  private static final String ROW3 = "row3";
-  private static final String COLF1 = "colf1";
-  private static final List<Entry<Key,Value>> row1;
-  private static final List<Entry<Key,Value>> row2;
-  private static final List<Entry<Key,Value>> row3;
-  private static AssertionError e1 = null;
-  private static AssertionError e2 = null;
-
-  static {
-    row1 = new ArrayList<Entry<Key,Value>>();
-    row1.add(new KeyValue(new Key(ROW1, COLF1, "colq1"), "v1".getBytes()));
-    row1.add(new KeyValue(new Key(ROW1, COLF1, "colq2"), "v2".getBytes()));
-    row1.add(new KeyValue(new Key(ROW1, "colf2", "colq3"), "v3".getBytes()));
-    row2 = new ArrayList<Entry<Key,Value>>();
-    row2.add(new KeyValue(new Key(ROW2, COLF1, "colq4"), "v4".getBytes()));
-    row3 = new ArrayList<Entry<Key,Value>>();
-    row3.add(new KeyValue(new Key(ROW3, COLF1, "colq5"), "v5".getBytes()));
-  }
-
-  public static void checkLists(final List<Entry<Key,Value>> first, final List<Entry<Key,Value>> second) {
-    assertEquals("Sizes should be the same.", first.size(), second.size());
-    for (int i = 0; i < first.size(); i++) {
-      assertEquals("Keys should be equal.", first.get(i).getKey(), second.get(i).getKey());
-      assertEquals("Values should be equal.", first.get(i).getValue(), second.get(i).getValue());
-    }
-  }
-
-  public static void checkLists(final List<Entry<Key,Value>> first, final Iterator<Entry<Key,Value>> second) {
-    int entryIndex = 0;
-    while (second.hasNext()) {
-      final Entry<Key,Value> entry = second.next();
-      assertEquals("Keys should be equal", first.get(entryIndex).getKey(), entry.getKey());
-      assertEquals("Values should be equal", first.get(entryIndex).getValue(), entry.getValue());
-      entryIndex++;
-    }
-  }
-
-  public static void insertList(final BatchWriter writer, final List<Entry<Key,Value>> list) throws MutationsRejectedException {
-    for (Entry<Key,Value> e : list) {
-      final Key key = e.getKey();
-      final Mutation mutation = new Mutation(key.getRow());
-      ColumnVisibility colVisibility = new ColumnVisibility(key.getColumnVisibility());
-      mutation.put(key.getColumnFamily(), key.getColumnQualifier(), colVisibility, key.getTimestamp(), e.getValue());
-      writer.addMutation(mutation);
-    }
-  }
-
-  private static class MRTester extends Configured implements Tool {
-    private static class TestMapper extends Mapper<Text,PeekingIterator<Entry<Key,Value>>,Key,Value> {
-      int count = 0;
-
-      @Override
-      protected void map(Text k, PeekingIterator<Entry<Key,Value>> v, Context context) throws IOException, InterruptedException {
-        try {
-          switch (count) {
-            case 0:
-              assertEquals("Current key should be " + ROW1, new Text(ROW1), k);
-              checkLists(row1, v);
-              break;
-            case 1:
-              assertEquals("Current key should be " + ROW2, new Text(ROW2), k);
-              checkLists(row2, v);
-              break;
-            case 2:
-              assertEquals("Current key should be " + ROW3, new Text(ROW3), k);
-              checkLists(row3, v);
-              break;
-            default:
-              assertTrue(false);
-          }
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        count++;
-      }
-
-      @Override
-      protected void cleanup(Context context) throws IOException, InterruptedException {
-        try {
-          assertEquals(3, count);
-        } catch (AssertionError e) {
-          e2 = e;
-        }
-      }
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 3) {
-        throw new IllegalArgumentException("Usage : " + MRTester.class.getName() + " <user> <pass> <table>");
-      }
-
-      String user = args[0];
-      String pass = args[1];
-      String table = args[2];
-
-      Job job = Job.getInstance(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormatClass(AccumuloRowInputFormat.class);
-
-      AccumuloInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
-      AccumuloInputFormat.setInputTableName(job, table);
-      AccumuloRowInputFormat.setMockInstance(job, INSTANCE_NAME);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormatClass(NullOutputFormat.class);
-
-      job.setNumReduceTasks(0);
-
-      job.waitForCompletion(true);
-
-      return job.isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTester(), args));
-    }
-  }
-
-  @Test
-  public void test() throws Exception {
-    final Instance instance = new org.apache.accumulo.core.client.mock.MockInstance(INSTANCE_NAME);
-    final Connector conn = instance.getConnector("root", new PasswordToken(""));
-    conn.tableOperations().create(TEST_TABLE_1);
-    BatchWriter writer = null;
-    try {
-      writer = conn.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
-      insertList(writer, row1);
-      insertList(writer, row2);
-      insertList(writer, row3);
-    } finally {
-      if (writer != null) {
-        writer.close();
-      }
-    }
-    MRTester.main(new String[] {"root", "", TEST_TABLE_1});
-    assertNull(e1);
-    assertNull(e2);
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/core/src/test/java/org/apache/accumulo/core/client/mapreduce/BadPasswordSplitsAccumuloInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/BadPasswordSplitsAccumuloInputFormat.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/BadPasswordSplitsAccumuloInputFormat.java
deleted file mode 100644
index 9028d94..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/BadPasswordSplitsAccumuloInputFormat.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-
-/**
- * AccumuloInputFormat which returns an "empty" RangeInputSplit
- */
-public class BadPasswordSplitsAccumuloInputFormat extends AccumuloInputFormat {
-
-  @Override
-  public List<InputSplit> getSplits(JobContext context) throws IOException {
-    List<InputSplit> splits = super.getSplits(context);
-
-    for (InputSplit split : splits) {
-      org.apache.accumulo.core.client.mapreduce.RangeInputSplit rangeSplit = (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) split;
-      rangeSplit.setToken(new PasswordToken("anythingelse"));
-    }
-
-    return splits;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/core/src/test/java/org/apache/accumulo/core/client/mapreduce/EmptySplitsAccumuloInputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/EmptySplitsAccumuloInputFormat.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/EmptySplitsAccumuloInputFormat.java
deleted file mode 100644
index dd531c0..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/EmptySplitsAccumuloInputFormat.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-
-/**
- * AccumuloInputFormat which returns an "empty" RangeInputSplit
- */
-public class EmptySplitsAccumuloInputFormat extends AccumuloInputFormat {
-
-  @Override
-  public List<InputSplit> getSplits(JobContext context) throws IOException {
-    List<InputSplit> oldSplits = super.getSplits(context);
-    List<InputSplit> newSplits = new ArrayList<InputSplit>(oldSplits.size());
-
-    // Copy only the necessary information
-    for (InputSplit oldSplit : oldSplits) {
-      org.apache.accumulo.core.client.mapreduce.RangeInputSplit newSplit = new org.apache.accumulo.core.client.mapreduce.RangeInputSplit(
-          (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) oldSplit);
-      newSplits.add(newSplit);
-    }
-
-    return newSplits;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
deleted file mode 100644
index 825d905..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/TokenFileTest.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.client.mapreduce;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.util.Iterator;
-import java.util.Map.Entry;
-
-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.Scanner;
-import org.apache.accumulo.core.client.impl.Credentials;
-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.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.util.CachedConfiguration;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-/**
- *
- */
-public class TokenFileTest {
-  private static AssertionError e1 = null;
-  private static final String PREFIX = TokenFileTest.class.getSimpleName();
-  private static final String INSTANCE_NAME = PREFIX + "_mapreduce_instance";
-  private static final String TEST_TABLE_1 = PREFIX + "_mapreduce_table_1";
-  private static final String TEST_TABLE_2 = PREFIX + "_mapreduce_table_2";
-
-  private static class MRTokenFileTester extends Configured implements Tool {
-    private static class TestMapper extends Mapper<Key,Value,Text,Mutation> {
-      Key key = null;
-      int count = 0;
-
-      @Override
-      protected void map(Key k, Value v, Context context) throws IOException, InterruptedException {
-        try {
-          if (key != null)
-            assertEquals(key.getRow().toString(), new String(v.get()));
-          assertEquals(k.getRow(), new Text(String.format("%09x", count + 1)));
-          assertEquals(new String(v.get()), String.format("%09x", count));
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        key = new Key(k);
-        count++;
-      }
-
-      @Override
-      protected void cleanup(Context context) throws IOException, InterruptedException {
-        Mutation m = new Mutation("total");
-        m.put("", "", Integer.toString(count));
-        context.write(new Text(), m);
-      }
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-
-      if (args.length != 4) {
-        throw new IllegalArgumentException("Usage : " + MRTokenFileTester.class.getName() + " <user> <token file> <inputtable> <outputtable>");
-      }
-
-      String user = args[0];
-      String tokenFile = args[1];
-      String table1 = args[2];
-      String table2 = args[3];
-
-      Job job = Job.getInstance(getConf(), this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormatClass(AccumuloInputFormat.class);
-
-      AccumuloInputFormat.setConnectorInfo(job, user, tokenFile);
-      AccumuloInputFormat.setInputTableName(job, table1);
-      AccumuloInputFormat.setMockInstance(job, INSTANCE_NAME);
-
-      job.setMapperClass(TestMapper.class);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormatClass(AccumuloOutputFormat.class);
-      job.setOutputKeyClass(Text.class);
-      job.setOutputValueClass(Mutation.class);
-
-      AccumuloOutputFormat.setConnectorInfo(job, user, tokenFile);
-      AccumuloOutputFormat.setCreateTables(job, false);
-      AccumuloOutputFormat.setDefaultTableName(job, table2);
-      AccumuloOutputFormat.setMockInstance(job, INSTANCE_NAME);
-
-      job.setNumReduceTasks(0);
-
-      job.waitForCompletion(true);
-
-      return job.isSuccessful() ? 0 : 1;
-    }
-
-    public static void main(String[] args) throws Exception {
-      Configuration conf = CachedConfiguration.getInstance();
-      conf.set("hadoop.tmp.dir", new File(args[1]).getParent());
-      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      assertEquals(0, ToolRunner.run(conf, new MRTokenFileTester(), args));
-    }
-  }
-
-  @Rule
-  public TemporaryFolder folder = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
-
-  @Test
-  public void testMR() throws Exception {
-    Instance mockInstance = new org.apache.accumulo.core.client.mock.MockInstance(INSTANCE_NAME);
-    Connector c = mockInstance.getConnector("root", new PasswordToken(""));
-    c.tableOperations().create(TEST_TABLE_1);
-    c.tableOperations().create(TEST_TABLE_2);
-    BatchWriter bw = c.createBatchWriter(TEST_TABLE_1, new BatchWriterConfig());
-    for (int i = 0; i < 100; i++) {
-      Mutation m = new Mutation(new Text(String.format("%09x", i + 1)));
-      m.put(new Text(), new Text(), new Value(String.format("%09x", i).getBytes()));
-      bw.addMutation(m);
-    }
-    bw.close();
-
-    File tf = folder.newFile("root_test.pw");
-    PrintStream out = new PrintStream(tf);
-    String outString = new Credentials("root", new PasswordToken("")).serialize();
-    out.println(outString);
-    out.close();
-
-    MRTokenFileTester.main(new String[] {"root", tf.getAbsolutePath(), TEST_TABLE_1, TEST_TABLE_2});
-    assertNull(e1);
-
-    Scanner scanner = c.createScanner(TEST_TABLE_2, new Authorizations());
-    Iterator<Entry<Key,Value>> iter = scanner.iterator();
-    assertTrue(iter.hasNext());
-    Entry<Key,Value> entry = iter.next();
-    assertEquals(Integer.parseInt(new String(entry.getValue().get())), 100);
-    assertFalse(iter.hasNext());
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java
index 286b343..700f93b 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java
@@ -16,30 +16,18 @@
  */
 package org.apache.accumulo.core.iterators.user;
 
-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.HashSet;
-import java.util.Iterator;
-import java.util.Map.Entry;
 import java.util.Random;
 import java.util.TreeMap;
 
-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.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.ByteSequence;
 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.DefaultIteratorEnvironment;
@@ -47,7 +35,6 @@ import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.accumulo.core.iterators.system.MultiIterator;
-import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
 import org.junit.Rule;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/examples/simple/src/main/java/org/apache/accumulo/examples/simple/dirlist/FileCount.java
----------------------------------------------------------------------
diff --git a/examples/simple/src/main/java/org/apache/accumulo/examples/simple/dirlist/FileCount.java b/examples/simple/src/main/java/org/apache/accumulo/examples/simple/dirlist/FileCount.java
index dabb4c1..111fae0 100644
--- a/examples/simple/src/main/java/org/apache/accumulo/examples/simple/dirlist/FileCount.java
+++ b/examples/simple/src/main/java/org/apache/accumulo/examples/simple/dirlist/FileCount.java
@@ -233,7 +233,7 @@ public class FileCount {
     }
   }
 
-  FileCount(Opts opts, ScannerOpts scanOpts, BatchWriterOpts bwOpts) throws Exception {
+  public FileCount(Opts opts, ScannerOpts scanOpts, BatchWriterOpts bwOpts) throws Exception {
     this.opts = opts;
     this.scanOpts = scanOpts;
     this.bwOpts = bwOpts;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/examples/simple/src/test/java/org/apache/accumulo/examples/simple/dirlist/CountTest.java
----------------------------------------------------------------------
diff --git a/examples/simple/src/test/java/org/apache/accumulo/examples/simple/dirlist/CountTest.java b/examples/simple/src/test/java/org/apache/accumulo/examples/simple/dirlist/CountTest.java
deleted file mode 100644
index 492f3e5..0000000
--- a/examples/simple/src/test/java/org/apache/accumulo/examples/simple/dirlist/CountTest.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.examples.simple.dirlist;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-
-import java.util.ArrayList;
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.cli.BatchWriterOpts;
-import org.apache.accumulo.core.cli.ClientOpts.Password;
-import org.apache.accumulo.core.cli.ScannerOpts;
-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.Scanner;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.examples.simple.dirlist.FileCount.Opts;
-import org.apache.hadoop.io.Text;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-public class CountTest {
-
-  @Rule
-  public TestName test = new TestName();
-
-  private Connector conn;
-  private String tableName;
-
-  @Before
-  public void setupInstance() throws Exception {
-    tableName = test.getMethodName();
-    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    conn = inst.getConnector("root", new PasswordToken(""));
-    conn.tableOperations().create(tableName);
-    BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig());
-    ColumnVisibility cv = new ColumnVisibility();
-    // / has 1 dir
-    // /local has 2 dirs 1 file
-    // /local/user1 has 2 files
-    bw.addMutation(Ingest.buildMutation(cv, "/local", true, false, true, 272, 12345, null));
-    bw.addMutation(Ingest.buildMutation(cv, "/local/user1", true, false, true, 272, 12345, null));
-    bw.addMutation(Ingest.buildMutation(cv, "/local/user2", true, false, true, 272, 12345, null));
-    bw.addMutation(Ingest.buildMutation(cv, "/local/file", false, false, false, 1024, 12345, null));
-    bw.addMutation(Ingest.buildMutation(cv, "/local/file", false, false, false, 1024, 23456, null));
-    bw.addMutation(Ingest.buildMutation(cv, "/local/user1/file1", false, false, false, 2024, 12345, null));
-    bw.addMutation(Ingest.buildMutation(cv, "/local/user1/file2", false, false, false, 1028, 23456, null));
-    bw.close();
-  }
-
-  @Test
-  public void test() throws Exception {
-    Scanner scanner = conn.createScanner(tableName, new Authorizations());
-    scanner.fetchColumn(new Text("dir"), new Text("counts"));
-    assertFalse(scanner.iterator().hasNext());
-
-    Opts opts = new Opts();
-    ScannerOpts scanOpts = new ScannerOpts();
-    BatchWriterOpts bwOpts = new BatchWriterOpts();
-    opts.instance = "counttest";
-    opts.setTableName(tableName);
-    opts.setPassword(new Password("secret"));
-    opts.mock = true;
-    opts.setPassword(new Opts.Password(""));
-    FileCount fc = new FileCount(opts, scanOpts, bwOpts);
-    fc.run();
-
-    ArrayList<Pair<String,String>> expected = new ArrayList<Pair<String,String>>();
-    expected.add(new Pair<String,String>(QueryUtil.getRow("").toString(), "1,0,3,3"));
-    expected.add(new Pair<String,String>(QueryUtil.getRow("/local").toString(), "2,1,2,3"));
-    expected.add(new Pair<String,String>(QueryUtil.getRow("/local/user1").toString(), "0,2,0,2"));
-    expected.add(new Pair<String,String>(QueryUtil.getRow("/local/user2").toString(), "0,0,0,0"));
-
-    int i = 0;
-    for (Entry<Key,Value> e : scanner) {
-      assertEquals(e.getKey().getRow().toString(), expected.get(i).getFirst());
-      assertEquals(e.getValue().toString(), expected.get(i).getSecond());
-      i++;
-    }
-    assertEquals(i, expected.size());
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
----------------------------------------------------------------------
diff --git a/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java b/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
deleted file mode 100644
index 997612f..0000000
--- a/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputFormatTest.java
+++ /dev/null
@@ -1,312 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.examples.simple.filedata;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map.Entry;
-
-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.security.tokens.PasswordToken;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-public class ChunkInputFormatTest {
-
-  private static AssertionError e0 = null;
-  private static AssertionError e1 = null;
-  private static AssertionError e2 = null;
-  private static IOException e3 = null;
-
-  private static final Authorizations AUTHS = new Authorizations("A", "B", "C", "D");
-
-  private static List<Entry<Key,Value>> data;
-  private static List<Entry<Key,Value>> baddata;
-
-  private Connector conn;
-
-  @Rule
-  public TestName test = new TestName();
-
-  @Before
-  public void setupInstance() throws Exception {
-    Instance instance = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    conn = instance.getConnector("root", new PasswordToken(""));
-  }
-
-  @BeforeClass
-  public static void setupClass() {
-    System.setProperty("hadoop.tmp.dir", System.getProperty("user.dir") + "/target/hadoop-tmp");
-
-    data = new ArrayList<Entry<Key,Value>>();
-    ChunkInputStreamTest.addData(data, "a", "refs", "ida\0ext", "A&B", "ext");
-    ChunkInputStreamTest.addData(data, "a", "refs", "ida\0name", "A&B", "name");
-    ChunkInputStreamTest.addData(data, "a", "~chunk", 100, 0, "A&B", "asdfjkl;");
-    ChunkInputStreamTest.addData(data, "a", "~chunk", 100, 1, "A&B", "");
-    ChunkInputStreamTest.addData(data, "b", "refs", "ida\0ext", "A&B", "ext");
-    ChunkInputStreamTest.addData(data, "b", "refs", "ida\0name", "A&B", "name");
-    ChunkInputStreamTest.addData(data, "b", "~chunk", 100, 0, "A&B", "qwertyuiop");
-    ChunkInputStreamTest.addData(data, "b", "~chunk", 100, 0, "B&C", "qwertyuiop");
-    ChunkInputStreamTest.addData(data, "b", "~chunk", 100, 1, "A&B", "");
-    ChunkInputStreamTest.addData(data, "b", "~chunk", 100, 1, "B&C", "");
-    ChunkInputStreamTest.addData(data, "b", "~chunk", 100, 1, "D", "");
-    baddata = new ArrayList<Entry<Key,Value>>();
-    ChunkInputStreamTest.addData(baddata, "c", "refs", "ida\0ext", "A&B", "ext");
-    ChunkInputStreamTest.addData(baddata, "c", "refs", "ida\0name", "A&B", "name");
-  }
-
-  public static void entryEquals(Entry<Key,Value> e1, Entry<Key,Value> e2) {
-    assertEquals(e1.getKey(), e2.getKey());
-    assertEquals(e1.getValue(), e2.getValue());
-  }
-
-  public static class CIFTester extends Configured implements Tool {
-    public static class TestMapper extends Mapper<List<Entry<Key,Value>>,InputStream,List<Entry<Key,Value>>,InputStream> {
-      int count = 0;
-
-      @Override
-      protected void map(List<Entry<Key,Value>> key, InputStream value, Context context) throws IOException, InterruptedException {
-        byte[] b = new byte[20];
-        int read;
-        try {
-          switch (count) {
-            case 0:
-              assertEquals(key.size(), 2);
-              entryEquals(key.get(0), data.get(0));
-              entryEquals(key.get(1), data.get(1));
-              assertEquals(read = value.read(b), 8);
-              assertEquals(new String(b, 0, read), "asdfjkl;");
-              assertEquals(read = value.read(b), -1);
-              break;
-            case 1:
-              assertEquals(key.size(), 2);
-              entryEquals(key.get(0), data.get(4));
-              entryEquals(key.get(1), data.get(5));
-              assertEquals(read = value.read(b), 10);
-              assertEquals(new String(b, 0, read), "qwertyuiop");
-              assertEquals(read = value.read(b), -1);
-              break;
-            default:
-              assertTrue(false);
-          }
-        } catch (AssertionError e) {
-          e1 = e;
-        } finally {
-          value.close();
-        }
-        count++;
-      }
-
-      @Override
-      protected void cleanup(Context context) throws IOException, InterruptedException {
-        try {
-          assertEquals(2, count);
-        } catch (AssertionError e) {
-          e2 = e;
-        }
-      }
-    }
-
-    public static class TestNoClose extends Mapper<List<Entry<Key,Value>>,InputStream,List<Entry<Key,Value>>,InputStream> {
-      int count = 0;
-
-      @Override
-      protected void map(List<Entry<Key,Value>> key, InputStream value, Context context) throws IOException, InterruptedException {
-        byte[] b = new byte[5];
-        int read;
-        try {
-          switch (count) {
-            case 0:
-              assertEquals(read = value.read(b), 5);
-              assertEquals(new String(b, 0, read), "asdfj");
-              break;
-            default:
-              assertTrue(false);
-          }
-        } catch (AssertionError e) {
-          e1 = e;
-        }
-        count++;
-        try {
-          context.nextKeyValue();
-          assertTrue(false);
-        } catch (IOException ioe) {
-          e3 = ioe;
-        }
-      }
-    }
-
-    public static class TestBadData extends Mapper<List<Entry<Key,Value>>,InputStream,List<Entry<Key,Value>>,InputStream> {
-      @Override
-      protected void map(List<Entry<Key,Value>> key, InputStream value, Context context) throws IOException, InterruptedException {
-        byte[] b = new byte[20];
-        try {
-          assertEquals(key.size(), 2);
-          entryEquals(key.get(0), baddata.get(0));
-          entryEquals(key.get(1), baddata.get(1));
-        } catch (AssertionError e) {
-          e0 = e;
-        }
-        try {
-          assertFalse(value.read(b) > 0);
-          try {
-            fail();
-          } catch (AssertionError e) {
-            e1 = e;
-          }
-        } catch (Exception e) {
-          // expected, ignore
-        }
-        try {
-          value.close();
-          try {
-            fail();
-          } catch (AssertionError e) {
-            e2 = e;
-          }
-        } catch (Exception e) {
-          // expected, ignore
-        }
-      }
-    }
-
-    @Override
-    public int run(String[] args) throws Exception {
-      if (args.length != 5) {
-        throw new IllegalArgumentException("Usage : " + CIFTester.class.getName() + " <instance name> <user> <pass> <table> <mapperClass>");
-      }
-
-      String instance = args[0];
-      String user = args[1];
-      String pass = args[2];
-      String table = args[3];
-
-      Job job = Job.getInstance(getConf());
-      job.setJobName(this.getClass().getSimpleName() + "_" + System.currentTimeMillis());
-      job.setJarByClass(this.getClass());
-
-      job.setInputFormatClass(ChunkInputFormat.class);
-
-      ChunkInputFormat.setConnectorInfo(job, user, new PasswordToken(pass));
-      ChunkInputFormat.setInputTableName(job, table);
-      ChunkInputFormat.setScanAuthorizations(job, AUTHS);
-      ChunkInputFormat.setMockInstance(job, instance);
-
-      @SuppressWarnings("unchecked")
-      Class<? extends Mapper<?,?,?,?>> forName = (Class<? extends Mapper<?,?,?,?>>) Class.forName(args[4]);
-      job.setMapperClass(forName);
-      job.setMapOutputKeyClass(Key.class);
-      job.setMapOutputValueClass(Value.class);
-      job.setOutputFormatClass(NullOutputFormat.class);
-
-      job.setNumReduceTasks(0);
-
-      job.waitForCompletion(true);
-
-      return job.isSuccessful() ? 0 : 1;
-    }
-
-    public static int main(String... args) throws Exception {
-      Configuration conf = new Configuration();
-      conf.set("mapreduce.cluster.local.dir", new File(System.getProperty("user.dir"), "target/mapreduce-tmp").getAbsolutePath());
-      return ToolRunner.run(conf, new CIFTester(), args);
-    }
-  }
-
-  @Test
-  public void test() throws Exception {
-    conn.tableOperations().create("test");
-    BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig());
-
-    for (Entry<Key,Value> e : data) {
-      Key k = e.getKey();
-      Mutation m = new Mutation(k.getRow());
-      m.put(k.getColumnFamily(), k.getColumnQualifier(), new ColumnVisibility(k.getColumnVisibility()), k.getTimestamp(), e.getValue());
-      bw.addMutation(m);
-    }
-    bw.close();
-
-    assertEquals(0, CIFTester.main(conn.getInstance().getInstanceName(), "root", "", "test", CIFTester.TestMapper.class.getName()));
-    assertNull(e1);
-    assertNull(e2);
-  }
-
-  @Test
-  public void testErrorOnNextWithoutClose() throws Exception {
-    conn.tableOperations().create("test");
-    BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig());
-
-    for (Entry<Key,Value> e : data) {
-      Key k = e.getKey();
-      Mutation m = new Mutation(k.getRow());
-      m.put(k.getColumnFamily(), k.getColumnQualifier(), new ColumnVisibility(k.getColumnVisibility()), k.getTimestamp(), e.getValue());
-      bw.addMutation(m);
-    }
-    bw.close();
-
-    assertEquals(1, CIFTester.main(conn.getInstance().getInstanceName(), "root", "", "test", CIFTester.TestNoClose.class.getName()));
-    assertNull(e1);
-    assertNull(e2);
-    assertNotNull(e3);
-  }
-
-  @Test
-  public void testInfoWithoutChunks() throws Exception {
-    conn.tableOperations().create("test");
-    BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig());
-    for (Entry<Key,Value> e : baddata) {
-      Key k = e.getKey();
-      Mutation m = new Mutation(k.getRow());
-      m.put(k.getColumnFamily(), k.getColumnQualifier(), new ColumnVisibility(k.getColumnVisibility()), k.getTimestamp(), e.getValue());
-      bw.addMutation(m);
-    }
-    bw.close();
-
-    assertEquals(0, CIFTester.main(conn.getInstance().getInstanceName(), "root", "", "test", CIFTester.TestBadData.class.getName()));
-    assertNull(e0);
-    assertNull(e1);
-    assertNull(e2);
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputStreamTest.java
----------------------------------------------------------------------
diff --git a/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputStreamTest.java b/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputStreamTest.java
index 614a480..17fbb76 100644
--- a/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputStreamTest.java
+++ b/examples/simple/src/test/java/org/apache/accumulo/examples/simple/filedata/ChunkInputStreamTest.java
@@ -27,36 +27,21 @@ import java.util.ArrayList;
 import java.util.List;
 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.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.Scanner;
-import org.apache.accumulo.core.client.TableExistsException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyValue;
-import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.util.PeekingIterator;
 import org.apache.hadoop.io.Text;
 import org.junit.Before;
-import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class ChunkInputStreamTest {
   private static final Logger log = LoggerFactory.getLogger(ChunkInputStream.class);
-  List<Entry<Key,Value>> data;
-  List<Entry<Key,Value>> baddata;
-  List<Entry<Key,Value>> multidata;
+  private List<Entry<Key,Value>> data;
+  private List<Entry<Key,Value>> baddata;
+  private List<Entry<Key,Value>> multidata;
 
   @Before
   public void setupData() {
@@ -103,11 +88,11 @@ public class ChunkInputStreamTest {
     addData(multidata, "c", "~chunk", 100, 1, "B&C", "");
   }
 
-  public static void addData(List<Entry<Key,Value>> data, String row, String cf, String cq, String vis, String value) {
+  private static void addData(List<Entry<Key,Value>> data, String row, String cf, String cq, String vis, String value) {
     data.add(new KeyValue(new Key(new Text(row), new Text(cf), new Text(cq), new Text(vis)), value.getBytes()));
   }
 
-  public static void addData(List<Entry<Key,Value>> data, String row, String cf, int chunkSize, int chunkCount, String vis, String value) {
+  private static void addData(List<Entry<Key,Value>> data, String row, String cf, int chunkSize, int chunkCount, String vis, String value) {
     Text chunkCQ = new Text(FileDataIngest.intToBytes(chunkSize));
     chunkCQ.append(FileDataIngest.intToBytes(chunkCount), 0, 4);
     data.add(new KeyValue(new Key(new Text(row), new Text(cf), chunkCQ, new Text(vis)), value.getBytes()));
@@ -235,63 +220,6 @@ public class ChunkInputStreamTest {
     assertFalse(pi.hasNext());
   }
 
-  @Rule
-  public TestName test = new TestName();
-
-  @Test
-  public void testWithAccumulo() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException, IOException {
-    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    Connector conn = inst.getConnector("root", new PasswordToken(""));
-    conn.tableOperations().create("test");
-    BatchWriter bw = conn.createBatchWriter("test", new BatchWriterConfig());
-
-    for (Entry<Key,Value> e : data) {
-      Key k = e.getKey();
-      Mutation m = new Mutation(k.getRow());
-      m.put(k.getColumnFamily(), k.getColumnQualifier(), new ColumnVisibility(k.getColumnVisibility()), e.getValue());
-      bw.addMutation(m);
-    }
-    bw.close();
-
-    Scanner scan = conn.createScanner("test", new Authorizations("A", "B", "C", "D"));
-
-    ChunkInputStream cis = new ChunkInputStream();
-    byte[] b = new byte[20];
-    int read;
-    PeekingIterator<Entry<Key,Value>> pi = new PeekingIterator<Entry<Key,Value>>(scan.iterator());
-
-    cis.setSource(pi);
-    assertEquals(read = cis.read(b), 8);
-    assertEquals(new String(b, 0, read), "asdfjkl;");
-    assertEquals(read = cis.read(b), -1);
-
-    cis.setSource(pi);
-    assertEquals(read = cis.read(b), 10);
-    assertEquals(new String(b, 0, read), "qwertyuiop");
-    assertEquals(read = cis.read(b), -1);
-    assertEquals(cis.getVisibilities().toString(), "[A&B, B&C, D]");
-    cis.close();
-
-    cis.setSource(pi);
-    assertEquals(read = cis.read(b), 16);
-    assertEquals(new String(b, 0, read), "asdfjkl;asdfjkl;");
-    assertEquals(read = cis.read(b), -1);
-    assertEquals(cis.getVisibilities().toString(), "[A&B]");
-    cis.close();
-
-    cis.setSource(pi);
-    assertEquals(read = cis.read(b), -1);
-    cis.close();
-
-    cis.setSource(pi);
-    assertEquals(read = cis.read(b), 8);
-    assertEquals(new String(b, 0, read), "asdfjkl;");
-    assertEquals(read = cis.read(b), -1);
-    cis.close();
-
-    assertFalse(pi.hasNext());
-  }
-
   private static void assumeExceptionOnRead(ChunkInputStream cis, byte[] b) {
     try {
       assertEquals(0, cis.read(b));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/server/base/src/test/java/org/apache/accumulo/server/util/TabletIteratorTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/TabletIteratorTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/TabletIteratorTest.java
deleted file mode 100644
index 19bf13a..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/util/TabletIteratorTest.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.server.util;
-
-import java.util.Map.Entry;
-
-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.Scanner;
-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.data.impl.KeyExtent;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.server.util.TabletIterator.TabletDeletedException;
-import org.apache.hadoop.io.Text;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.TestName;
-
-public class TabletIteratorTest {
-
-  @Rule
-  public TestName test = new TestName();
-
-  @Rule
-  public ExpectedException exception = ExpectedException.none();
-
-  class TestTabletIterator extends TabletIterator {
-
-    private Connector conn;
-
-    public TestTabletIterator(Connector conn) throws Exception {
-      super(conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY), MetadataSchema.TabletsSection.getRange(), true, true);
-      this.conn = conn;
-    }
-
-    @Override
-    protected void resetScanner() {
-      try {
-        Scanner ds = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-        Text tablet = new KeyExtent(new Text("0"), new Text("m"), null).getMetadataEntry();
-        ds.setRange(new Range(tablet, true, tablet, true));
-
-        Mutation m = new Mutation(tablet);
-
-        BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-        for (Entry<Key,Value> entry : ds) {
-          Key k = entry.getKey();
-          m.putDelete(k.getColumnFamily(), k.getColumnQualifier(), k.getTimestamp());
-        }
-
-        bw.addMutation(m);
-
-        bw.close();
-
-      } catch (Exception e) {
-        throw new RuntimeException(e);
-      }
-
-      super.resetScanner();
-    }
-
-  }
-
-  // simulate a merge happening while iterating over tablets
-  @Test
-  public void testMerge() throws Exception {
-    Instance mi = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    Connector conn = mi.getConnector("", new PasswordToken(""));
-
-    KeyExtent ke1 = new KeyExtent(new Text("0"), new Text("m"), null);
-    Mutation mut1 = ke1.getPrevRowUpdateMutation();
-    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(mut1, new Value("/d1".getBytes()));
-
-    KeyExtent ke2 = new KeyExtent(new Text("0"), null, null);
-    Mutation mut2 = ke2.getPrevRowUpdateMutation();
-    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(mut2, new Value("/d2".getBytes()));
-
-    BatchWriter bw1 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    bw1.addMutation(mut1);
-    bw1.addMutation(mut2);
-    bw1.close();
-
-    TestTabletIterator tabIter = new TestTabletIterator(conn);
-
-    exception.expect(TabletDeletedException.class);
-    while (tabIter.hasNext()) {
-      tabIter.next();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
----------------------------------------------------------------------
diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
deleted file mode 100644
index 92a72fb..0000000
--- a/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.gc.replication;
-
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map.Entry;
-import java.util.Set;
-
-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.Scanner;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.conf.SiteConfiguration;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.server.AccumuloServerContext;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.hadoop.io.Text;
-import org.easymock.EasyMock;
-import org.easymock.IAnswer;
-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.Iterables;
-
-public class CloseWriteAheadLogReferencesTest {
-
-  private CloseWriteAheadLogReferences refs;
-  private Connector conn;
-
-  @Rule
-  public TestName testName = new TestName();
-
-  @Before
-  public void setupInstance() throws Exception {
-    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(testName.getMethodName());
-    conn = inst.getConnector("root", new PasswordToken(""));
-  }
-
-  @Before
-  public void setup() {
-    Instance mockInst = createMock(Instance.class);
-    SiteConfiguration siteConfig = EasyMock.createMock(SiteConfiguration.class);
-    expect(mockInst.getInstanceID()).andReturn(testName.getMethodName()).anyTimes();
-    expect(mockInst.getZooKeepers()).andReturn("localhost").anyTimes();
-    expect(mockInst.getZooKeepersSessionTimeOut()).andReturn(30000).anyTimes();
-    final AccumuloConfiguration systemConf = new ConfigurationCopy(new HashMap<String,String>());
-    ServerConfigurationFactory factory = createMock(ServerConfigurationFactory.class);
-    expect(factory.getConfiguration()).andReturn(systemConf).anyTimes();
-    expect(factory.getInstance()).andReturn(mockInst).anyTimes();
-    expect(factory.getSiteConfiguration()).andReturn(siteConfig).anyTimes();
-
-    // Just make the SiteConfiguration delegate to our AccumuloConfiguration
-    // Presently, we only need get(Property) and iterator().
-    EasyMock.expect(siteConfig.get(EasyMock.anyObject(Property.class))).andAnswer(new IAnswer<String>() {
-      @Override
-      public String answer() {
-        Object[] args = EasyMock.getCurrentArguments();
-        return systemConf.get((Property) args[0]);
-      }
-    }).anyTimes();
-    EasyMock.expect(siteConfig.getBoolean(EasyMock.anyObject(Property.class))).andAnswer(new IAnswer<Boolean>() {
-      @Override
-      public Boolean answer() {
-        Object[] args = EasyMock.getCurrentArguments();
-        return systemConf.getBoolean((Property) args[0]);
-      }
-    }).anyTimes();
-
-    EasyMock.expect(siteConfig.iterator()).andAnswer(new IAnswer<Iterator<Entry<String,String>>>() {
-      @Override
-      public Iterator<Entry<String,String>> answer() {
-        return systemConf.iterator();
-      }
-    }).anyTimes();
-
-    replay(mockInst, factory, siteConfig);
-    refs = new CloseWriteAheadLogReferences(new AccumuloServerContext(factory));
-  }
-
-  @Test
-  public void unclosedWalsLeaveStatusOpen() throws Exception {
-    Set<String> wals = Collections.emptySet();
-    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    Mutation m = new Mutation(ReplicationSection.getRowPrefix() + "file:/accumulo/wal/tserver+port/12345");
-    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
-    bw.addMutation(m);
-    bw.close();
-
-    refs.updateReplicationEntries(conn, wals);
-
-    Scanner s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
-    Status status = Status.parseFrom(entry.getValue().get());
-    Assert.assertFalse(status.getClosed());
-  }
-
-  @Test
-  public void closedWalsUpdateStatus() throws Exception {
-    String file = "file:/accumulo/wal/tserver+port/12345";
-    Set<String> wals = Collections.singleton(file);
-    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
-    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
-    bw.addMutation(m);
-    bw.close();
-
-    refs.updateReplicationEntries(conn, wals);
-
-    Scanner s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
-    Status status = Status.parseFrom(entry.getValue().get());
-    Assert.assertTrue(status.getClosed());
-  }
-
-  @Test
-  public void partiallyReplicatedReferencedWalsAreNotClosed() throws Exception {
-    String file = "file:/accumulo/wal/tserver+port/12345";
-    Set<String> wals = Collections.singleton(file);
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
-    Mutation m = new Mutation(file);
-    StatusSection.add(m, new Text("1"), ProtobufUtil.toValue(StatusUtil.ingestedUntil(1000)));
-    bw.addMutation(m);
-    bw.close();
-
-    refs.updateReplicationEntries(conn, wals);
-
-    Scanner s = ReplicationTable.getScanner(conn);
-    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
-    Status status = Status.parseFrom(entry.getValue().get());
-    Assert.assertFalse(status.getClosed());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
index 7f23ad5..7d8f406 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
@@ -16,6 +16,8 @@
  */
 package org.apache.accumulo.master;
 
+import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
+
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -96,15 +98,14 @@ import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NoNodeException;
 import org.slf4j.Logger;
 
-import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 import com.google.protobuf.InvalidProtocolBufferException;
 
-class MasterClientServiceHandler extends FateServiceHandler implements MasterClientService.Iface {
+public class MasterClientServiceHandler extends FateServiceHandler implements MasterClientService.Iface {
 
   private static final Logger log = Master.log;
   private Instance instance;
 
-  MasterClientServiceHandler(Master master) {
+  protected MasterClientServiceHandler(Master master) {
     super(master);
     this.instance = master.getInstance();
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java b/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java
deleted file mode 100644
index a18e5e9..0000000
--- a/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java
+++ /dev/null
@@ -1,451 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.master;
-
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.ClientConfiguration;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.TableNotFoundException;
-import org.apache.accumulo.core.client.impl.ClientContext;
-import org.apache.accumulo.core.client.impl.Credentials;
-import org.apache.accumulo.core.client.impl.ReplicationOperationsImpl;
-import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
-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.Value;
-import org.apache.accumulo.core.data.impl.KeyExtent;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.thrift.TCredentials;
-import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.hadoop.io.Text;
-import org.apache.thrift.TException;
-import org.easymock.EasyMock;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class ReplicationOperationsImplTest {
-  private static final Logger log = LoggerFactory.getLogger(ReplicationOperationsImplTest.class);
-
-  private Instance inst;
-
-  @Rule
-  public TestName test = new TestName();
-
-  @Before
-  public void setup() {
-    inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-  }
-
-  /**
-   * Spoof out the Master so we can call the implementation without starting a full instance.
-   */
-  private ReplicationOperationsImpl getReplicationOperations(ClientContext context) throws Exception {
-    Master master = EasyMock.createMock(Master.class);
-    EasyMock.expect(master.getConnector()).andReturn(inst.getConnector("root", new PasswordToken(""))).anyTimes();
-    EasyMock.expect(master.getInstance()).andReturn(inst).anyTimes();
-    EasyMock.replay(master);
-
-    final MasterClientServiceHandler mcsh = new MasterClientServiceHandler(master) {
-      @Override
-      protected String getTableId(Instance inst, String tableName) throws ThriftTableOperationException {
-        try {
-          return inst.getConnector("root", new PasswordToken("")).tableOperations().tableIdMap().get(tableName);
-        } catch (Exception e) {
-          throw new RuntimeException(e);
-        }
-      }
-    };
-
-    return new ReplicationOperationsImpl(context) {
-      @Override
-      protected boolean getMasterDrain(final TInfo tinfo, final TCredentials rpcCreds, final String tableName, final Set<String> wals)
-          throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-        try {
-          return mcsh.drainReplicationTable(tinfo, rpcCreds, tableName, wals);
-        } catch (TException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    };
-  }
-
-  @Test
-  public void waitsUntilEntriesAreReplicated() throws Exception {
-    Connector conn = inst.getConnector("root", new PasswordToken(""));
-    conn.tableOperations().create("foo");
-    Text tableId = new Text(conn.tableOperations().tableIdMap().get("foo"));
-
-    String file1 = "/accumulo/wals/tserver+port/" + UUID.randomUUID(), file2 = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    Status stat = Status.newBuilder().setBegin(0).setEnd(10000).setInfiniteEnd(false).setClosed(false).build();
-
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
-
-    Mutation m = new Mutation(file1);
-    StatusSection.add(m, tableId, ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-
-    m = new Mutation(file2);
-    StatusSection.add(m, tableId, ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-
-    bw.close();
-
-    bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
-    m.put(ReplicationSection.COLF, tableId, ProtobufUtil.toValue(stat));
-
-    bw.addMutation(m);
-
-    m = new Mutation(ReplicationSection.getRowPrefix() + file2);
-    m.put(ReplicationSection.COLF, tableId, ProtobufUtil.toValue(stat));
-
-    bw.close();
-
-    final AtomicBoolean done = new AtomicBoolean(false);
-    final AtomicBoolean exception = new AtomicBoolean(false);
-    ClientContext context = new ClientContext(inst, new Credentials("root", new PasswordToken("")), new ClientConfiguration());
-    final ReplicationOperationsImpl roi = getReplicationOperations(context);
-    Thread t = new Thread(new Runnable() {
-      @Override
-      public void run() {
-        try {
-          roi.drain("foo");
-        } catch (Exception e) {
-          log.error("Got error", e);
-          exception.set(true);
-        }
-        done.set(true);
-      }
-    });
-
-    t.start();
-
-    // With the records, we shouldn't be drained
-    Assert.assertFalse(done.get());
-
-    bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
-    m.putDelete(ReplicationSection.COLF, tableId);
-    bw.addMutation(m);
-    bw.flush();
-
-    Assert.assertFalse(done.get());
-
-    m = new Mutation(ReplicationSection.getRowPrefix() + file2);
-    m.putDelete(ReplicationSection.COLF, tableId);
-    bw.addMutation(m);
-    bw.flush();
-    bw.close();
-
-    // Removing metadata entries doesn't change anything
-    Assert.assertFalse(done.get());
-
-    // Remove the replication entries too
-    bw = ReplicationTable.getBatchWriter(conn);
-    m = new Mutation(file1);
-    m.putDelete(StatusSection.NAME, tableId);
-    bw.addMutation(m);
-    bw.flush();
-
-    Assert.assertFalse(done.get());
-
-    m = new Mutation(file2);
-    m.putDelete(StatusSection.NAME, tableId);
-    bw.addMutation(m);
-    bw.flush();
-
-    try {
-      t.join(5000);
-    } catch (InterruptedException e) {
-      Assert.fail("ReplicationOperations.drain did not complete");
-    }
-
-    // After both metadata and replication
-    Assert.assertTrue("Drain never finished", done.get());
-    Assert.assertFalse("Saw unexpectetd exception", exception.get());
-  }
-
-  @Test
-  public void unrelatedReplicationRecordsDontBlockDrain() throws Exception {
-    Connector conn = inst.getConnector("root", new PasswordToken(""));
-    conn.tableOperations().create("foo");
-    conn.tableOperations().create("bar");
-
-    Text tableId1 = new Text(conn.tableOperations().tableIdMap().get("foo"));
-    Text tableId2 = new Text(conn.tableOperations().tableIdMap().get("bar"));
-
-    String file1 = "/accumulo/wals/tserver+port/" + UUID.randomUUID(), file2 = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    Status stat = Status.newBuilder().setBegin(0).setEnd(10000).setInfiniteEnd(false).setClosed(false).build();
-
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
-
-    Mutation m = new Mutation(file1);
-    StatusSection.add(m, tableId1, ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-
-    m = new Mutation(file2);
-    StatusSection.add(m, tableId2, ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-
-    bw.close();
-
-    bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
-    m.put(ReplicationSection.COLF, tableId1, ProtobufUtil.toValue(stat));
-
-    bw.addMutation(m);
-
-    m = new Mutation(ReplicationSection.getRowPrefix() + file2);
-    m.put(ReplicationSection.COLF, tableId2, ProtobufUtil.toValue(stat));
-
-    bw.close();
-
-    final AtomicBoolean done = new AtomicBoolean(false);
-    final AtomicBoolean exception = new AtomicBoolean(false);
-    ClientContext context = new ClientContext(inst, new Credentials("root", new PasswordToken("")), new ClientConfiguration());
-
-    final ReplicationOperationsImpl roi = getReplicationOperations(context);
-
-    Thread t = new Thread(new Runnable() {
-      @Override
-      public void run() {
-        try {
-          roi.drain("foo");
-        } catch (Exception e) {
-          log.error("Got error", e);
-          exception.set(true);
-        }
-        done.set(true);
-      }
-    });
-
-    t.start();
-
-    // With the records, we shouldn't be drained
-    Assert.assertFalse(done.get());
-
-    bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
-    m.putDelete(ReplicationSection.COLF, tableId1);
-    bw.addMutation(m);
-    bw.flush();
-
-    // Removing metadata entries doesn't change anything
-    Assert.assertFalse(done.get());
-
-    // Remove the replication entries too
-    bw = ReplicationTable.getBatchWriter(conn);
-    m = new Mutation(file1);
-    m.putDelete(StatusSection.NAME, tableId1);
-    bw.addMutation(m);
-    bw.flush();
-
-    try {
-      t.join(5000);
-    } catch (InterruptedException e) {
-      Assert.fail("ReplicationOperations.drain did not complete");
-    }
-
-    // After both metadata and replication
-    Assert.assertTrue("Drain never completed", done.get());
-    Assert.assertFalse("Saw unexpected exception", exception.get());
-  }
-
-  @Test
-  public void inprogressReplicationRecordsBlockExecution() throws Exception {
-    Connector conn = inst.getConnector("root", new PasswordToken(""));
-    conn.tableOperations().create("foo");
-
-    Text tableId1 = new Text(conn.tableOperations().tableIdMap().get("foo"));
-
-    String file1 = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    Status stat = Status.newBuilder().setBegin(0).setEnd(10000).setInfiniteEnd(false).setClosed(false).build();
-
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
-
-    Mutation m = new Mutation(file1);
-    StatusSection.add(m, tableId1, ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-    bw.close();
-
-    LogEntry logEntry = new LogEntry(new KeyExtent(new Text(tableId1), null, null), System.currentTimeMillis(), "tserver", file1);
-
-    bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
-    m.put(ReplicationSection.COLF, tableId1, ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-
-    m = new Mutation(logEntry.getRow());
-    m.put(logEntry.getColumnFamily(), logEntry.getColumnQualifier(), logEntry.getValue());
-    bw.addMutation(m);
-
-    bw.close();
-
-    final AtomicBoolean done = new AtomicBoolean(false);
-    final AtomicBoolean exception = new AtomicBoolean(false);
-    ClientContext context = new ClientContext(inst, new Credentials("root", new PasswordToken("")), new ClientConfiguration());
-    final ReplicationOperationsImpl roi = getReplicationOperations(context);
-    Thread t = new Thread(new Runnable() {
-      @Override
-      public void run() {
-        try {
-          roi.drain("foo");
-        } catch (Exception e) {
-          log.error("Got error", e);
-          exception.set(true);
-        }
-        done.set(true);
-      }
-    });
-
-    t.start();
-
-    // With the records, we shouldn't be drained
-    Assert.assertFalse(done.get());
-
-    Status newStatus = Status.newBuilder().setBegin(1000).setEnd(2000).setInfiniteEnd(false).setClosed(true).build();
-    bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
-    m.put(ReplicationSection.COLF, tableId1, ProtobufUtil.toValue(newStatus));
-    bw.addMutation(m);
-    bw.flush();
-
-    // Removing metadata entries doesn't change anything
-    Assert.assertFalse(done.get());
-
-    // Remove the replication entries too
-    bw = ReplicationTable.getBatchWriter(conn);
-    m = new Mutation(file1);
-    m.put(StatusSection.NAME, tableId1, ProtobufUtil.toValue(newStatus));
-    bw.addMutation(m);
-    bw.flush();
-
-    try {
-      t.join(5000);
-    } catch (InterruptedException e) {
-      Assert.fail("ReplicationOperations.drain did not complete");
-    }
-
-    // New records, but not fully replicated ones don't cause it to complete
-    Assert.assertFalse("Drain somehow finished", done.get());
-    Assert.assertFalse("Saw unexpected exception", exception.get());
-  }
-
-  @Test
-  public void laterCreatedLogsDontBlockExecution() throws Exception {
-    Connector conn = inst.getConnector("root", new PasswordToken(""));
-    conn.tableOperations().create("foo");
-
-    Text tableId1 = new Text(conn.tableOperations().tableIdMap().get("foo"));
-
-    String file1 = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    Status stat = Status.newBuilder().setBegin(0).setEnd(10000).setInfiniteEnd(false).setClosed(false).build();
-
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
-    Mutation m = new Mutation(file1);
-    StatusSection.add(m, tableId1, ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-    bw.close();
-
-    bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
-    m.put(ReplicationSection.COLF, tableId1, ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-
-    bw.close();
-
-    System.out.println("Reading metadata first time");
-    for (Entry<Key,Value> e : conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-      System.out.println(e.getKey());
-    }
-
-    final AtomicBoolean done = new AtomicBoolean(false);
-    final AtomicBoolean exception = new AtomicBoolean(false);
-    ClientContext context = new ClientContext(inst, new Credentials("root", new PasswordToken("")), new ClientConfiguration());
-    final ReplicationOperationsImpl roi = getReplicationOperations(context);
-    Thread t = new Thread(new Runnable() {
-      @Override
-      public void run() {
-        try {
-          roi.drain("foo");
-        } catch (Exception e) {
-          log.error("Got error", e);
-          exception.set(true);
-        }
-        done.set(true);
-      }
-    });
-
-    t.start();
-
-    // We need to wait long enough for the table to read once
-    Thread.sleep(2000);
-
-    // Write another file, but also delete the old files
-    bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    m = new Mutation(ReplicationSection.getRowPrefix() + "/accumulo/wals/tserver+port/" + UUID.randomUUID());
-    m.put(ReplicationSection.COLF, tableId1, ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-    m = new Mutation(ReplicationSection.getRowPrefix() + file1);
-    m.putDelete(ReplicationSection.COLF, tableId1);
-    bw.addMutation(m);
-    bw.close();
-
-    System.out.println("Reading metadata second time");
-    for (Entry<Key,Value> e : conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
-      System.out.println(e.getKey());
-    }
-
-    bw = ReplicationTable.getBatchWriter(conn);
-    m = new Mutation(file1);
-    m.putDelete(StatusSection.NAME, tableId1);
-    bw.addMutation(m);
-    bw.close();
-
-    try {
-      t.join(5000);
-    } catch (InterruptedException e) {
-      Assert.fail("ReplicationOperations.drain did not complete");
-    }
-
-    // We should pass immediately because we aren't waiting on both files to be deleted (just the one that we did)
-    Assert.assertTrue("Drain didn't finish", done.get());
-  }
-
-}


[08/14] accumulo git commit: ACCUMULO-3920 moved some test away from mock

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c62e5f/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java b/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java
index 271822d..6189ee3 100644
--- a/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java
+++ b/test/src/test/java/org/apache/accumulo/test/iterator/RegExTest.java
@@ -17,38 +17,29 @@
 package org.apache.accumulo.test.iterator;
 
 import java.util.ArrayList;
-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.Instance;
+import java.util.Set;
+import java.util.TreeMap;
+
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.client.ScannerBase;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.data.ByteSequence;
 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.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.accumulo.core.iterators.user.RegExFilter;
-import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import com.google.common.collect.ImmutableSet;
+
 public class RegExTest {
 
-  private static Connector conn;
+  private static TreeMap<Key,Value> data = new TreeMap<>();
 
   @BeforeClass
   public static void setupTests() throws Exception {
-    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(RegExTest.class.getName());
-    conn = inst.getConnector("user", new PasswordToken("pass"));
-    conn.tableOperations().create("ret");
-    BatchWriter bw = conn.createBatchWriter("ret", new BatchWriterConfig());
 
     ArrayList<Character> chars = new ArrayList<Character>();
     for (char c = 'a'; c <= 'z'; c++)
@@ -59,18 +50,14 @@ public class RegExTest {
 
     // insert some data into accumulo
     for (Character rc : chars) {
-      Mutation m = new Mutation(new Text("r" + rc));
+      String row = "r" + rc;
       for (Character cfc : chars) {
         for (Character cqc : chars) {
           Value v = new Value(("v" + rc + cfc + cqc).getBytes());
-          m.put(new Text("cf" + cfc), new Text("cq" + cqc), v);
+          data.put(new Key(row, "cf" + cfc, "cq" + cqc, "", 9), v);
         }
       }
-
-      bw.addMutation(m);
     }
-
-    bw.close();
   }
 
   private void check(String regex, String val) throws Exception {
@@ -123,42 +110,29 @@ public class RegExTest {
 
   private void runTest(Range range, String rowRegEx, String cfRegEx, String cqRegEx, String valRegEx, int expected) throws Exception {
 
-    Scanner s = conn.createScanner("ret", Authorizations.EMPTY);
-    s.setRange(range);
-    setRegexs(s, rowRegEx, cfRegEx, cqRegEx, valRegEx);
-    runTest(s, rowRegEx, cfRegEx, cqRegEx, valRegEx, expected);
-
-    BatchScanner bs = conn.createBatchScanner("ret", Authorizations.EMPTY, 1);
-    bs.setRanges(Collections.singletonList(range));
-    setRegexs(bs, rowRegEx, cfRegEx, cqRegEx, valRegEx);
-    runTest(bs, rowRegEx, cfRegEx, cqRegEx, valRegEx, expected);
-    bs.close();
-  }
-
-  private void setRegexs(ScannerBase scanner, String rowRegEx, String cfRegEx, String cqRegEx, String valRegEx) {
-    IteratorSetting regex = new IteratorSetting(50, "regex", RegExFilter.class);
-    if (rowRegEx != null)
-      regex.addOption(RegExFilter.ROW_REGEX, rowRegEx);
-    if (cfRegEx != null)
-      regex.addOption(RegExFilter.COLF_REGEX, cfRegEx);
-    if (cqRegEx != null)
-      regex.addOption(RegExFilter.COLQ_REGEX, cqRegEx);
-    if (valRegEx != null)
-      regex.addOption(RegExFilter.VALUE_REGEX, valRegEx);
-    scanner.addScanIterator(regex);
+    SortedKeyValueIterator<Key,Value> source = new SortedMapIterator(data);
+    Set<ByteSequence> es = ImmutableSet.of();
+    IteratorSetting is = new IteratorSetting(50, "regex", RegExFilter.class);
+    RegExFilter.setRegexs(is, rowRegEx, cfRegEx, cqRegEx, valRegEx, false);
+    RegExFilter iter = new RegExFilter();
+    iter.init(source, is.getOptions(), null);
+    iter.seek(range, es, false);
+    runTest(iter, rowRegEx, cfRegEx, cqRegEx, valRegEx, expected);
   }
 
-  private void runTest(Iterable<Entry<Key,Value>> scanner, String rowRegEx, String cfRegEx, String cqRegEx, String valRegEx, int expected) throws Exception {
+  private void runTest(RegExFilter scanner, String rowRegEx, String cfRegEx, String cqRegEx, String valRegEx, int expected) throws Exception {
 
     int counter = 0;
 
-    for (Entry<Key,Value> entry : scanner) {
-      Key k = entry.getKey();
+    while (scanner.hasTop()) {
+      Key k = scanner.getTopKey();
 
       check(rowRegEx, k.getRow());
       check(cfRegEx, k.getColumnFamily());
       check(cqRegEx, k.getColumnQualifier());
-      check(valRegEx, entry.getValue());
+      check(valRegEx, scanner.getTopValue());
+
+      scanner.next();
 
       counter++;
     }


[09/14] accumulo git commit: ACCUMULO-3920 moved some test away from mock

Posted by ct...@apache.org.
ACCUMULO-3920 moved some test away from mock


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

Branch: refs/heads/master
Commit: 66c62e5f357a50044da0ab4a56fccb527ed6ac2c
Parents: 2556df3
Author: Keith Turner <ke...@deenlo.com>
Authored: Thu Jul 9 16:25:10 2015 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Thu Jul 30 16:33:03 2015 -0400

----------------------------------------------------------------------
 .../accumulo/core/client/admin/FindMaxTest.java | 119 ------
 .../user/IntersectingIteratorTest.java          |  26 --
 .../core/iterators/user/RegExFilterTest.java    |  49 +--
 .../core/iterators/user/RowFilterTest.java      | 128 +++---
 .../user/TransformingIteratorTest.java          | 202 +++++-----
 .../accumulo/server/util/MetadataTableUtil.java |  29 +-
 .../master/balancer/TableLoadBalancerTest.java  |  39 +-
 .../server/security/SystemCredentialsTest.java  |   5 +-
 .../apache/accumulo/server/util/CloneTest.java  | 376 ------------------
 .../accumulo/shell/ShellSetInstanceTest.java    |   7 +-
 .../java/org/apache/accumulo/test/CloneIT.java  | 390 +++++++++++++++++++
 .../org/apache/accumulo/test/FindMaxIT.java     | 113 ++++++
 .../accumulo/test/iterator/RegExTest.java       |  76 ++--
 13 files changed, 748 insertions(+), 811 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c62e5f/core/src/test/java/org/apache/accumulo/core/client/admin/FindMaxTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/admin/FindMaxTest.java b/core/src/test/java/org/apache/accumulo/core/client/admin/FindMaxTest.java
deleted file mode 100644
index 78f7e75..0000000
--- a/core/src/test/java/org/apache/accumulo/core/client/admin/FindMaxTest.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.core.client.admin;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
-import java.util.ArrayList;
-import java.util.Map.Entry;
-
-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.Scanner;
-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.Value;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.hadoop.io.Text;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-public class FindMaxTest {
-
-  private static Mutation nm(byte[] row) {
-    Mutation m = new Mutation(new Text(row));
-    m.put("cf", "cq", "v");
-    return m;
-  }
-
-  private static Mutation nm(String row) {
-    Mutation m = new Mutation(row);
-    m.put("cf", "cq", "v");
-    return m;
-  }
-
-  @Rule
-  public TestName test = new TestName();
-
-  @Test
-  public void test1() throws Exception {
-    Instance mi = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-
-    Connector conn = mi.getConnector("root", new PasswordToken(""));
-    conn.tableOperations().create("foo");
-
-    BatchWriter bw = conn.createBatchWriter("foo", new BatchWriterConfig());
-
-    bw.addMutation(nm(new byte[] {0}));
-    bw.addMutation(nm(new byte[] {0, 0}));
-    bw.addMutation(nm(new byte[] {0, 1}));
-    bw.addMutation(nm(new byte[] {0, 1, 0}));
-    bw.addMutation(nm(new byte[] {1, 0}));
-    bw.addMutation(nm(new byte[] {'a', 'b', 'c'}));
-    bw.addMutation(nm(new byte[] {(byte) 0xff}));
-    bw.addMutation(nm(new byte[] {(byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff}));
-
-    for (int i = 0; i < 1000; i++) {
-      bw.addMutation(nm(String.format("r%05d", i)));
-    }
-
-    bw.close();
-
-    Scanner scanner = conn.createScanner("foo", Authorizations.EMPTY);
-
-    ArrayList<Text> rows = new ArrayList<Text>();
-
-    for (Entry<Key,Value> entry : scanner) {
-      rows.add(entry.getKey().getRow());
-    }
-
-    for (int i = rows.size() - 1; i > 0; i--) {
-      Text max = FindMax.findMax(conn.createScanner("foo", Authorizations.EMPTY), null, true, rows.get(i), false);
-      assertEquals(rows.get(i - 1), max);
-
-      max = FindMax.findMax(conn.createScanner("foo", Authorizations.EMPTY), rows.get(i - 1), true, rows.get(i), false);
-      assertEquals(rows.get(i - 1), max);
-
-      max = FindMax.findMax(conn.createScanner("foo", Authorizations.EMPTY), rows.get(i - 1), false, rows.get(i), false);
-      assertNull(max);
-
-      max = FindMax.findMax(conn.createScanner("foo", Authorizations.EMPTY), null, true, rows.get(i), true);
-      assertEquals(rows.get(i), max);
-
-      max = FindMax.findMax(conn.createScanner("foo", Authorizations.EMPTY), rows.get(i), true, rows.get(i), true);
-      assertEquals(rows.get(i), max);
-
-      max = FindMax.findMax(conn.createScanner("foo", Authorizations.EMPTY), rows.get(i - 1), false, rows.get(i), true);
-      assertEquals(rows.get(i), max);
-
-    }
-
-    Text max = FindMax.findMax(conn.createScanner("foo", Authorizations.EMPTY), null, true, null, true);
-    assertEquals(rows.get(rows.size() - 1), max);
-
-    max = FindMax.findMax(conn.createScanner("foo", Authorizations.EMPTY), null, true, new Text(new byte[] {0}), false);
-    assertNull(max);
-
-    max = FindMax.findMax(conn.createScanner("foo", Authorizations.EMPTY), null, true, new Text(new byte[] {0}), true);
-    assertEquals(rows.get(0), max);
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c62e5f/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java
index c3696d4..286b343 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest.java
@@ -295,30 +295,4 @@ public class IntersectingIteratorTest {
     assertTrue(hitCount == docs.size());
     cleanup();
   }
-
-  @Test
-  public void testWithBatchScanner() throws Exception {
-    Value empty = new Value(new byte[] {});
-    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    Connector connector = inst.getConnector("user", new PasswordToken("pass"));
-    connector.tableOperations().create("index");
-    BatchWriter bw = connector.createBatchWriter("index", new BatchWriterConfig());
-    Mutation m = new Mutation("000012");
-    m.put("rvy", "5000000000000000", empty);
-    m.put("15qh", "5000000000000000", empty);
-    bw.addMutation(m);
-    bw.close();
-
-    BatchScanner bs = connector.createBatchScanner("index", Authorizations.EMPTY, 10);
-    IteratorSetting ii = new IteratorSetting(20, IntersectingIterator.class);
-    IntersectingIterator.setColumnFamilies(ii, new Text[] {new Text("rvy"), new Text("15qh")});
-    bs.addScanIterator(ii);
-    bs.setRanges(Collections.singleton(new Range()));
-    Iterator<Entry<Key,Value>> iterator = bs.iterator();
-    assertTrue(iterator.hasNext());
-    Entry<Key,Value> next = iterator.next();
-    Key key = next.getKey();
-    assertEquals(key.getColumnQualifier(), new Text("5000000000000000"));
-    assertFalse(iterator.hasNext());
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c62e5f/core/src/test/java/org/apache/accumulo/core/iterators/user/RegExFilterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/RegExFilterTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/RegExFilterTest.java
index 7a203c8..61acc09 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/RegExFilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/RegExFilterTest.java
@@ -25,46 +25,20 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.TreeMap;
 
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
-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.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.ByteSequence;
 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.DefaultIteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
-import org.apache.accumulo.core.security.Authorizations;
 import org.apache.hadoop.io.Text;
-import org.junit.Before;
-import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TestName;
 
 public class RegExFilterTest {
 
   private static final Collection<ByteSequence> EMPTY_COL_FAMS = new ArrayList<ByteSequence>();
 
-  private Connector conn;
-
-  @Rule
-  public TestName test = new TestName();
-
-  @Before
-  public void setupInstance() throws Exception {
-    Instance instance = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    conn = instance.getConnector("root", new PasswordToken(""));
-  }
-
   private Key nkv(TreeMap<Key,Value> tm, String row, String cf, String cq, String val) {
     Key k = nk(row, cf, cq);
     tm.put(k, new Value(val.getBytes()));
@@ -267,8 +241,8 @@ public class RegExFilterTest {
   }
 
   @Test
-  public void testNullByteInKey() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException {
-    String table = "nullRegexTest";
+  public void testNullByteInKey() throws IOException {
+    TreeMap<Key,Value> tm = new TreeMap<Key,Value>();
 
     String s1 = "first", s2 = "second";
     byte[] b1 = s1.getBytes(), b2 = s2.getBytes(), ball;
@@ -277,22 +251,17 @@ public class RegExFilterTest {
     ball[b1.length] = (byte) 0;
     System.arraycopy(b2, 0, ball, b1.length + 1, b2.length);
 
-    conn.tableOperations().create(table);
-    BatchWriter bw = conn.createBatchWriter(table, new BatchWriterConfig());
-    Mutation m = new Mutation(ball);
-    m.put(new byte[0], new byte[0], new byte[0]);
-    bw.addMutation(m);
-    bw.close();
+    Key key = new Key(ball, new byte[0], new byte[0], new byte[0], 90, false);
+    Value val = new Value(new byte[0]);
+    tm.put(key, val);
 
     IteratorSetting is = new IteratorSetting(5, RegExFilter.class);
     RegExFilter.setRegexs(is, s2, null, null, null, true, true);
 
-    Scanner scanner = conn.createScanner(table, new Authorizations());
-    scanner.addScanIterator(is);
-
-    assertTrue("Client side iterator couldn't find a match when it should have", scanner.iterator().hasNext());
+    RegExFilter filter = new RegExFilter();
+    filter.init(new SortedMapIterator(tm), is.getOptions(), null);
+    filter.seek(new Range(), EMPTY_COL_FAMS, false);
 
-    conn.tableOperations().attachIterator(table, is);
-    assertTrue("server side iterator couldn't find a match when it should have", conn.createScanner(table, new Authorizations()).iterator().hasNext());
+    assertTrue("iterator couldn't find a match when it should have", filter.hasTop());
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c62e5f/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
index 6026c14..f0892b2 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/RowFilterTest.java
@@ -25,17 +25,10 @@ import java.util.Collections;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
-import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeMap;
 
-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.Scanner;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.ColumnUpdate;
 import org.apache.accumulo.core.data.Key;
@@ -45,25 +38,13 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.DefaultIteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.SortedMapIterator;
-import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator;
 import org.apache.hadoop.io.Text;
-import org.junit.Before;
-import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TestName;
 
-public class RowFilterTest {
-
-  @Rule
-  public TestName test = new TestName();
+import com.google.common.collect.ImmutableSet;
 
-  private Connector conn;
-
-  @Before
-  public void setupInstance() throws Exception {
-    Instance instance = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    conn = instance.getConnector("", new PasswordToken(""));
-  }
+public class RowFilterTest {
 
   public static class SummingRowFilter extends RowFilter {
 
@@ -84,7 +65,7 @@ public class RowFilterTest {
       }
 
       // ensure that seeks are confined to the row
-      rowIterator.seek(new Range(), new HashSet<ByteSequence>(), false);
+      rowIterator.seek(new Range(null, false, firstKey == null ? null : firstKey.getRow(), false), new HashSet<ByteSequence>(), false);
       while (rowIterator.hasTop()) {
         sum2 += Integer.parseInt(rowIterator.getTopValue().toString());
         rowIterator.next();
@@ -96,7 +77,7 @@ public class RowFilterTest {
         rowIterator.next();
       }
 
-      return sum == 2 && sum2 == 2;
+      return sum == 2 && sum2 == 0;
     }
 
   }
@@ -144,7 +125,7 @@ public class RowFilterTest {
 
     m = new Mutation("1");
     m.put("cf1", "cq1", "1");
-    m.put("cf1", "cq2", "2");
+    m.put("cf2", "cq2", "2");
     mutations.add(m);
 
     m = new Mutation("2");
@@ -154,7 +135,7 @@ public class RowFilterTest {
 
     m = new Mutation("3");
     m.put("cf1", "cq1", "0");
-    m.put("cf1", "cq2", "2");
+    m.put("cf2", "cq2", "2");
     mutations.add(m);
 
     m = new Mutation("4");
@@ -197,69 +178,63 @@ public class RowFilterTest {
 
   @Test
   public void test1() throws Exception {
-    conn.tableOperations().create("table1");
-    BatchWriter bw = conn.createBatchWriter("table1", new BatchWriterConfig());
+    ColumnFamilySkippingIterator source = new ColumnFamilySkippingIterator(new SortedMapIterator(createKeyValues()));
 
-    for (Mutation m : createMutations()) {
-      bw.addMutation(m);
-    }
-    IteratorSetting is = new IteratorSetting(40, SummingRowFilter.class);
-    conn.tableOperations().attachIterator("table1", is);
+    RowFilter filter = new SummingRowFilter();
+    filter.init(source, Collections.<String,String> emptyMap(), new DefaultIteratorEnvironment());
 
-    Scanner scanner = conn.createScanner("table1", Authorizations.EMPTY);
-    assertEquals(new HashSet<String>(Arrays.asList("2", "3")), getRows(scanner));
+    filter.seek(new Range(), Collections.<ByteSequence> emptySet(), false);
+
+    assertEquals(new HashSet<String>(Arrays.asList("2", "3")), getRows(filter));
 
-    scanner.fetchColumn(new Text("cf1"), new Text("cq2"));
-    assertEquals(new HashSet<String>(Arrays.asList("1", "3")), getRows(scanner));
+    ByteSequence cf = new ArrayByteSequence("cf2");
 
-    scanner.clearColumns();
-    scanner.fetchColumn(new Text("cf1"), new Text("cq1"));
-    assertEquals(new HashSet<String>(), getRows(scanner));
+    filter.seek(new Range(), ImmutableSet.of(cf), true);
+    assertEquals(new HashSet<String>(Arrays.asList("1", "3", "0", "4")), getRows(filter));
 
-    scanner.setRange(new Range("0", "4"));
-    scanner.clearColumns();
-    assertEquals(new HashSet<String>(Arrays.asList("2", "3")), getRows(scanner));
+    filter.seek(new Range("0", "4"), Collections.<ByteSequence> emptySet(), false);
+    assertEquals(new HashSet<String>(Arrays.asList("2", "3")), getRows(filter));
 
-    scanner.setRange(new Range("2"));
-    scanner.clearColumns();
-    assertEquals(new HashSet<String>(Arrays.asList("2")), getRows(scanner));
+    filter.seek(new Range("2"), Collections.<ByteSequence> emptySet(), false);
+    assertEquals(new HashSet<String>(Arrays.asList("2")), getRows(filter));
 
-    scanner.setRange(new Range("4"));
-    scanner.clearColumns();
-    assertEquals(new HashSet<String>(), getRows(scanner));
+    filter.seek(new Range("4"), Collections.<ByteSequence> emptySet(), false);
+    assertEquals(new HashSet<String>(), getRows(filter));
 
-    scanner.setRange(new Range("4"));
-    scanner.clearColumns();
-    scanner.fetchColumn(new Text("cf1"), new Text("cq2"));
-    scanner.fetchColumn(new Text("cf1"), new Text("cq4"));
-    assertEquals(new HashSet<String>(Arrays.asList("4")), getRows(scanner));
+    filter.seek(new Range("4"), ImmutableSet.of(cf), true);
+    assertEquals(new HashSet<String>(Arrays.asList("4")), getRows(filter));
 
   }
 
   @Test
   public void testChainedRowFilters() throws Exception {
-    conn.tableOperations().create("chained_row_filters");
-    BatchWriter bw = conn.createBatchWriter("chained_row_filters", new BatchWriterConfig());
-    for (Mutation m : createMutations()) {
-      bw.addMutation(m);
-    }
-    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(40, "trueFilter1", TrueFilter.class));
-    conn.tableOperations().attachIterator("chained_row_filters", new IteratorSetting(41, "trueFilter2", TrueFilter.class));
-    Scanner scanner = conn.createScanner("chained_row_filters", Authorizations.EMPTY);
-    assertEquals(new HashSet<String>(Arrays.asList("0", "1", "2", "3", "4")), getRows(scanner));
+    SortedMapIterator source = new SortedMapIterator(createKeyValues());
+
+    RowFilter filter0 = new TrueFilter();
+    filter0.init(source, Collections.<String,String> emptyMap(), new DefaultIteratorEnvironment());
+
+    RowFilter filter = new TrueFilter();
+    filter.init(filter0, Collections.<String,String> emptyMap(), new DefaultIteratorEnvironment());
+
+    filter.seek(new Range(), Collections.<ByteSequence> emptySet(), false);
+
+    assertEquals(new HashSet<String>(Arrays.asList("0", "1", "2", "3", "4")), getRows(filter));
   }
 
   @Test
   public void testFilterConjunction() throws Exception {
-    conn.tableOperations().create("filter_conjunction");
-    BatchWriter bw = conn.createBatchWriter("filter_conjunction", new BatchWriterConfig());
-    for (Mutation m : createMutations()) {
-      bw.addMutation(m);
-    }
-    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(40, "rowZeroOrOne", RowZeroOrOneFilter.class));
-    conn.tableOperations().attachIterator("filter_conjunction", new IteratorSetting(41, "rowOneOrTwo", RowOneOrTwoFilter.class));
-    Scanner scanner = conn.createScanner("filter_conjunction", Authorizations.EMPTY);
-    assertEquals(new HashSet<String>(Arrays.asList("1")), getRows(scanner));
+
+    SortedMapIterator source = new SortedMapIterator(createKeyValues());
+
+    RowFilter filter0 = new RowZeroOrOneFilter();
+    filter0.init(source, Collections.<String,String> emptyMap(), new DefaultIteratorEnvironment());
+
+    RowFilter filter = new RowOneOrTwoFilter();
+    filter.init(filter0, Collections.<String,String> emptyMap(), new DefaultIteratorEnvironment());
+
+    filter.seek(new Range(), Collections.<ByteSequence> emptySet(), false);
+
+    assertEquals(new HashSet<String>(Arrays.asList("1")), getRows(filter));
   }
 
   @Test
@@ -308,10 +283,11 @@ public class RowFilterTest {
     assertTrue("Expected next key read to be greater than the previous after deepCopy", lastKeyRead.compareTo(finalKeyRead) < 0);
   }
 
-  private HashSet<String> getRows(Scanner scanner) {
+  private HashSet<String> getRows(RowFilter filter) throws IOException {
     HashSet<String> rows = new HashSet<String>();
-    for (Entry<Key,Value> entry : scanner) {
-      rows.add(entry.getKey().getRow().toString());
+    while (filter.hasTop()) {
+      rows.add(filter.getTopKey().getRowData().toString());
+      filter.next();
     }
     return rows;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c62e5f/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java b/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
index e98afd7..1f4d6e7 100644
--- a/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/iterators/user/TransformingIteratorTest.java
@@ -24,83 +24,85 @@ 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.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
-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.Scanner;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.PartialKey;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
+import org.apache.accumulo.core.iterators.SortedMapIterator;
 import org.apache.accumulo.core.iterators.WrappingIterator;
+import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator;
+import org.apache.accumulo.core.iterators.system.VisibilityFilter;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.hadoop.io.Text;
+import org.easymock.EasyMock;
 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.ImmutableMap;
 
 public class TransformingIteratorTest {
-  private static final String TABLE_NAME = "test_table";
+
   private static Authorizations authorizations = new Authorizations("vis0", "vis1", "vis2", "vis3", "vis4");
-  private Connector connector;
-  private Scanner scanner;
+  private static final Map<String,String> EMPTY_OPTS = ImmutableMap.of();
+  private TransformingIterator titer;
 
-  @Rule
-  public TestName test = new TestName();
+  private TreeMap<Key,Value> data = new TreeMap<Key,Value>();
 
   @Before
-  public void setUpMockAccumulo() throws Exception {
-    Instance instance = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    connector = instance.getConnector("user", new PasswordToken("password"));
-    connector.securityOperations().changeUserAuthorizations("user", authorizations);
-
-    if (connector.tableOperations().exists(TABLE_NAME))
-      connector.tableOperations().delete(TABLE_NAME);
-    connector.tableOperations().create(TABLE_NAME);
-    BatchWriterConfig bwCfg = new BatchWriterConfig();
-    bwCfg.setMaxWriteThreads(1);
+  public void createData() throws Exception {
+    data.clear();
+    generateRow(data, "row1");
+    generateRow(data, "row2");
+    generateRow(data, "row3");
+  }
 
-    BatchWriter bw = connector.createBatchWriter(TABLE_NAME, bwCfg);
-    bw.addMutation(createDefaultMutation("row1"));
-    bw.addMutation(createDefaultMutation("row2"));
-    bw.addMutation(createDefaultMutation("row3"));
+  private void setUpTransformIterator(Class<? extends TransformingIterator> clazz) throws IOException {
+    setUpTransformIterator(clazz, true);
+  }
 
-    bw.flush();
-    bw.close();
+  private void setUpTransformIterator(Class<? extends TransformingIterator> clazz, boolean setupAuths) throws IOException {
+    SortedMapIterator source = new SortedMapIterator(data);
+    ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(source);
+    VisibilityFilter visFilter = new VisibilityFilter(cfsi, authorizations, new byte[0]);
+    ReuseIterator reuserIter = new ReuseIterator();
+    reuserIter.init(visFilter, EMPTY_OPTS, null);
+    try {
+      titer = clazz.newInstance();
+    } catch (InstantiationException | IllegalAccessException e) {
+      throw new RuntimeException(e);
+    }
 
-    scanner = connector.createScanner(TABLE_NAME, authorizations);
-    scanner.addScanIterator(new IteratorSetting(20, ReuseIterator.class));
-  }
+    IteratorEnvironment iterEnv = EasyMock.createMock(IteratorEnvironment.class);
+    EasyMock.expect(iterEnv.getIteratorScope()).andReturn(IteratorScope.scan).anyTimes();
+    EasyMock.replay(iterEnv);
 
-  private void setUpTransformIterator(Class<? extends TransformingIterator> clazz) {
-    IteratorSetting cfg = new IteratorSetting(21, clazz);
-    cfg.setName("keyTransformIter");
-    TransformingIterator.setAuthorizations(cfg, new Authorizations("vis0", "vis1", "vis2", "vis3"));
-    scanner.addScanIterator(cfg);
+    Map<String,String> opts;
+    if (setupAuths) {
+      IteratorSetting cfg = new IteratorSetting(21, clazz);
+      TransformingIterator.setAuthorizations(cfg, new Authorizations("vis0", "vis1", "vis2", "vis3"));
+      opts = cfg.getOptions();
+    } else {
+      opts = ImmutableMap.of();
+    }
+    titer.init(reuserIter, opts, iterEnv);
   }
 
   @Test
@@ -134,7 +136,6 @@ public class TransformingIteratorTest {
 
     // Test transforming col fam, col qual, col vis
     for (Class<? extends ReversingKeyTransformingIterator> clazz : classes) {
-      scanner.removeScanIterator("keyTransformIter");
       setUpTransformIterator(clazz);
 
       // All rows with visibilities reversed
@@ -164,7 +165,6 @@ public class TransformingIteratorTest {
     // Source data has vis1, vis2, vis3 so vis0 is a new one that is introduced.
     // Make sure it shows up in the output with the default test auths which include
     // vis0.
-    scanner.removeScanIterator("keyTransformIter");
     setUpTransformIterator(ColVisReversingKeyTransformingIterator.class);
     TreeMap<Key,Value> expected = new TreeMap<Key,Value>();
     for (int row = 1; row <= 3; ++row) {
@@ -182,13 +182,10 @@ public class TransformingIteratorTest {
   @Test
   public void testCreatingIllegalVisbility() throws Exception {
     // illegal visibility created by transform should be filtered on scan, even if evaluation is done
-    IteratorSetting cfg = new IteratorSetting(21, IllegalVisKeyTransformingIterator.class);
-    cfg.setName("keyTransformIter");
-    scanner.addScanIterator(cfg);
+    setUpTransformIterator(IllegalVisKeyTransformingIterator.class, false);
     checkExpected(new TreeMap<Key,Value>());
 
     // ensure illegal vis is supressed when evaluations is done
-    scanner.removeScanIterator("keyTransformIter");
     setUpTransformIterator(IllegalVisKeyTransformingIterator.class);
     checkExpected(new TreeMap<Key,Value>());
   }
@@ -196,26 +193,24 @@ public class TransformingIteratorTest {
   @Test
   public void testRangeStart() throws Exception {
     setUpTransformIterator(ColVisReversingKeyTransformingIterator.class);
-    scanner.setRange(new Range(new Key("row1", "cf2", "cq2", "vis1"), true, new Key("row1", "cf2", "cq3"), false));
 
     TreeMap<Key,Value> expected = new TreeMap<Key,Value>();
     putExpected(expected, 1, 2, 2, 1, PartialKey.ROW_COLFAM_COLQUAL); // before the range start, but transforms in the range
     putExpected(expected, 1, 2, 2, 2, PartialKey.ROW_COLFAM_COLQUAL);
 
-    checkExpected(expected);
+    checkExpected(new Range(new Key("row1", "cf2", "cq2", "vis1"), true, new Key("row1", "cf2", "cq3"), false), expected);
   }
 
   @Test
   public void testRangeEnd() throws Exception {
     setUpTransformIterator(ColVisReversingKeyTransformingIterator.class);
-    scanner.setRange(new Range(new Key("row1", "cf2", "cq2"), true, new Key("row1", "cf2", "cq2", "vis2"), false));
 
     TreeMap<Key,Value> expected = new TreeMap<Key,Value>();
     // putExpected(expected, 1, 2, 2, 1, part); // transforms vis outside range end
     putExpected(expected, 1, 2, 2, 2, PartialKey.ROW_COLFAM_COLQUAL);
     putExpected(expected, 1, 2, 2, 3, PartialKey.ROW_COLFAM_COLQUAL);
 
-    checkExpected(expected);
+    checkExpected(new Range(new Key("row1", "cf2", "cq2"), true, new Key("row1", "cf2", "cq2", "vis2"), false), expected);
   }
 
   @Test
@@ -224,13 +219,12 @@ public class TransformingIteratorTest {
     // Set a range that is before all of the untransformed data. However,
     // the data with untransformed col fam cf3 will transform to cf0 and
     // be inside the range.
-    scanner.setRange(new Range(new Key("row1", "cf0"), true, new Key("row1", "cf1"), false));
 
     TreeMap<Key,Value> expected = new TreeMap<Key,Value>();
     for (int cq = 1; cq <= 3; ++cq)
       for (int cv = 1; cv <= 3; ++cv)
         putExpected(expected, 1, 3, cq, cv, PartialKey.ROW);
-    checkExpected(expected);
+    checkExpected(new Range(new Key("row1", "cf0"), true, new Key("row1", "cf1"), false), expected);
   }
 
   @Test
@@ -238,8 +232,7 @@ public class TransformingIteratorTest {
     // Set a range that's after all data and make sure we don't
     // somehow return something.
     setUpTransformIterator(ColFamReversingKeyTransformingIterator.class);
-    scanner.setRange(new Range(new Key("row4"), null));
-    checkExpected(new TreeMap<Key,Value>());
+    checkExpected(new Range(new Key("row4"), null), new TreeMap<Key,Value>());
   }
 
   @Test
@@ -272,53 +265,47 @@ public class TransformingIteratorTest {
     // put in the expectations.
     int expectedCF = 1;
     setUpTransformIterator(ColFamReversingKeyTransformingIterator.class);
-    scanner.fetchColumnFamily(new Text("cf2"));
 
     TreeMap<Key,Value> expected = new TreeMap<Key,Value>();
     for (int row = 1; row <= 3; ++row)
       for (int cq = 1; cq <= 3; ++cq)
         for (int cv = 1; cv <= 3; ++cv)
           putExpected(expected, row, expectedCF, cq, cv, PartialKey.ROW);
-    checkExpected(expected);
+    checkExpected(expected, "cf2");
   }
 
   @Test
   public void testDeepCopy() throws Exception {
-    connector.tableOperations().create("shard_table");
-
-    BatchWriter bw = connector.createBatchWriter("shard_table", new BatchWriterConfig());
-
     ColumnVisibility vis1 = new ColumnVisibility("vis1");
     ColumnVisibility vis3 = new ColumnVisibility("vis3");
 
-    Mutation m1 = new Mutation("shard001");
-    m1.put("foo", "doc02", vis1, "");
-    m1.put("dog", "doc02", vis3, "");
-    m1.put("cat", "doc02", vis3, "");
+    data.clear();
 
-    m1.put("bar", "doc03", vis1, "");
-    m1.put("dog", "doc03", vis3, "");
-    m1.put("cat", "doc03", vis3, "");
+    Value ev = new Value("".getBytes());
 
-    bw.addMutation(m1);
-    bw.close();
+    data.put(new Key("shard001", "foo", "doc02", vis1, 78), ev);
+    data.put(new Key("shard001", "dog", "doc02", vis3, 78), ev);
+    data.put(new Key("shard001", "cat", "doc02", vis3, 78), ev);
 
-    BatchScanner bs = connector.createBatchScanner("shard_table", authorizations, 1);
+    data.put(new Key("shard001", "bar", "doc03", vis1, 78), ev);
+    data.put(new Key("shard001", "dog", "doc03", vis3, 78), ev);
+    data.put(new Key("shard001", "cat", "doc03", vis3, 78), ev);
+
+    setUpTransformIterator(ColVisReversingKeyTransformingIterator.class);
 
-    bs.addScanIterator(new IteratorSetting(21, ColVisReversingKeyTransformingIterator.class));
+    IntersectingIterator iiIter = new IntersectingIterator();
     IteratorSetting iicfg = new IteratorSetting(22, IntersectingIterator.class);
     IntersectingIterator.setColumnFamilies(iicfg, new Text[] {new Text("foo"), new Text("dog"), new Text("cat")});
-    bs.addScanIterator(iicfg);
-    bs.setRanges(Collections.singleton(new Range()));
+    iiIter.init(titer, iicfg.getOptions(), null);
 
-    Iterator<Entry<Key,Value>> iter = bs.iterator();
-    assertTrue(iter.hasNext());
-    Key docKey = iter.next().getKey();
+    iiIter.seek(new Range(), new HashSet<ByteSequence>(), false);
+
+    assertTrue(iiIter.hasTop());
+    Key docKey = iiIter.getTopKey();
     assertEquals("shard001", docKey.getRowData().toString());
     assertEquals("doc02", docKey.getColumnQualifierData().toString());
-    assertFalse(iter.hasNext());
-
-    bs.close();
+    iiIter.next();
+    assertFalse(iiIter.hasTop());
   }
 
   @Test
@@ -329,14 +316,13 @@ public class TransformingIteratorTest {
     // put in the expectations.
     int expectedCF = 1;
     setUpTransformIterator(ColFamReversingCompactionKeyTransformingIterator.class);
-    scanner.fetchColumnFamily(new Text("cf2"));
 
     TreeMap<Key,Value> expected = new TreeMap<Key,Value>();
     for (int row = 1; row <= 3; ++row)
       for (int cq = 1; cq <= 3; ++cq)
         for (int cv = 1; cv <= 3; ++cv)
           putExpected(expected, row, expectedCF, cq, cv, PartialKey.ROW);
-    checkExpected(expected);
+    checkExpected(expected, "cf2");
   }
 
   @Test
@@ -381,9 +367,12 @@ public class TransformingIteratorTest {
   public void testDupes() throws Exception {
     setUpTransformIterator(DupeTransformingIterator.class);
 
+    titer.seek(new Range(), new HashSet<ByteSequence>(), false);
+
     int count = 0;
-    for (Entry<Key,Value> entry : scanner) {
-      Key key = entry.getKey();
+    while (titer.hasTop()) {
+      Key key = titer.getTopKey();
+      titer.next();
       assertEquals("cf1", key.getColumnFamily().toString());
       assertEquals("cq1", key.getColumnQualifier().toString());
       assertEquals("", key.getColumnVisibility().toString());
@@ -429,13 +418,31 @@ public class TransformingIteratorTest {
     return key;
   }
 
-  private void checkExpected(TreeMap<Key,Value> expectedEntries) {
-    for (Entry<Key,Value> entry : scanner) {
+  private void checkExpected(Range range, TreeMap<Key,Value> expectedEntries) throws IOException {
+    checkExpected(range, new HashSet<ByteSequence>(), expectedEntries);
+  }
+
+  private void checkExpected(TreeMap<Key,Value> expectedEntries, String... fa) throws IOException {
+
+    HashSet<ByteSequence> families = new HashSet<>();
+    for (String family : fa) {
+      families.add(new ArrayByteSequence(family));
+    }
+
+    checkExpected(new Range(), families, expectedEntries);
+  }
+
+  private void checkExpected(Range range, Set<ByteSequence> families, TreeMap<Key,Value> expectedEntries) throws IOException {
+
+    titer.seek(range, families, families.size() != 0);
+
+    while (titer.hasTop()) {
       Entry<Key,Value> expected = expectedEntries.pollFirstEntry();
-      Key actualKey = entry.getKey();
-      Value actualValue = entry.getValue();
+      Key actualKey = titer.getTopKey();
+      Value actualValue = titer.getTopValue();
+      titer.next();
 
-      assertNotNull("Ran out of expected entries on: " + entry, expected);
+      assertNotNull("Ran out of expected entries on: " + actualKey, expected);
       assertEquals("Key mismatch", expected.getKey(), actualKey);
       assertEquals("Value mismatch", expected.getValue(), actualValue);
     }
@@ -480,8 +487,8 @@ public class TransformingIteratorTest {
     return new Text(sb.toString());
   }
 
-  private static Mutation createDefaultMutation(String row) {
-    Mutation m = new Mutation(row);
+  private static void generateRow(TreeMap<Key,Value> data, String row) {
+
     for (int cfID = 1; cfID <= 3; ++cfID) {
       for (int cqID = 1; cqID <= 3; ++cqID) {
         for (int cvID = 1; cvID <= 3; ++cvID) {
@@ -491,11 +498,13 @@ public class TransformingIteratorTest {
           long ts = 100 * cfID + 10 * cqID + cvID;
           String val = "val" + ts;
 
-          m.put(cf, cq, new ColumnVisibility(cv), ts, val);
+          Key k = new Key(row, cf, cq, cv, ts);
+          Value v = new Value(val.getBytes());
+          data.put(k, v);
         }
       }
     }
-    return m;
+
   }
 
   private static Key reverseKeyPart(Key originalKey, PartialKey part) {
@@ -666,6 +675,13 @@ public class TransformingIteratorTest {
     private Value topValue = new Value();
 
     @Override
+    public SortedKeyValueIterator<Key,Value> deepCopy(IteratorEnvironment env) {
+      ReuseIterator rei = new ReuseIterator();
+      rei.setSource(getSource().deepCopy(env));
+      return rei;
+    }
+
+    @Override
     public void seek(Range range, Collection<ByteSequence> columnFamilies, boolean inclusive) throws IOException {
       super.seek(range, columnFamilies, inclusive);
       loadTop();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c62e5f/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
index bcfbbc8..d4ec4f8 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
@@ -94,6 +94,7 @@ import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
 import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
@@ -706,10 +707,7 @@ public class MetadataTableUtil {
     return m;
   }
 
-  private static Scanner createCloneScanner(String tableId, Connector conn) throws TableNotFoundException {
-    String tableName = MetadataTable.NAME;
-    if (tableId.equals(MetadataTable.ID))
-      tableName = RootTable.NAME;
+  private static Scanner createCloneScanner(String tableName, String tableId, Connector conn) throws TableNotFoundException {
     Scanner mscanner = new IsolatedScanner(conn.createScanner(tableName, Authorizations.EMPTY));
     mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
     mscanner.fetchColumnFamily(DataFileColumnFamily.NAME);
@@ -721,12 +719,14 @@ public class MetadataTableUtil {
     return mscanner;
   }
 
-  static void initializeClone(String srcTableId, String tableId, Connector conn, BatchWriter bw) throws TableNotFoundException, MutationsRejectedException {
+  @VisibleForTesting
+  public static void initializeClone(String tableName, String srcTableId, String tableId, Connector conn, BatchWriter bw) throws TableNotFoundException,
+      MutationsRejectedException {
     TabletIterator ti;
     if (srcTableId.equals(MetadataTable.ID))
-      ti = new TabletIterator(createCloneScanner(srcTableId, conn), new Range(), true, true);
+      ti = new TabletIterator(createCloneScanner(tableName, srcTableId, conn), new Range(), true, true);
     else
-      ti = new TabletIterator(createCloneScanner(srcTableId, conn), new KeyExtent(new Text(srcTableId), null, null).toMetadataRange(), true, true);
+      ti = new TabletIterator(createCloneScanner(tableName, srcTableId, conn), new KeyExtent(new Text(srcTableId), null, null).toMetadataRange(), true, true);
 
     if (!ti.hasNext())
       throw new RuntimeException(" table deleted during clone?  srcTableId = " + srcTableId);
@@ -741,10 +741,13 @@ public class MetadataTableUtil {
     return new KeyExtent(new Text("0"), endRow1, null).compareTo(new KeyExtent(new Text("0"), endRow2, null));
   }
 
-  static int checkClone(String srcTableId, String tableId, Connector conn, BatchWriter bw) throws TableNotFoundException, MutationsRejectedException {
-    TabletIterator srcIter = new TabletIterator(createCloneScanner(srcTableId, conn), new KeyExtent(new Text(srcTableId), null, null).toMetadataRange(), true,
-        true);
-    TabletIterator cloneIter = new TabletIterator(createCloneScanner(tableId, conn), new KeyExtent(new Text(tableId), null, null).toMetadataRange(), true, true);
+  @VisibleForTesting
+  public static int checkClone(String tableName, String srcTableId, String tableId, Connector conn, BatchWriter bw) throws TableNotFoundException,
+      MutationsRejectedException {
+    TabletIterator srcIter = new TabletIterator(createCloneScanner(tableName, srcTableId, conn),
+        new KeyExtent(new Text(srcTableId), null, null).toMetadataRange(), true, true);
+    TabletIterator cloneIter = new TabletIterator(createCloneScanner(tableName, tableId, conn), new KeyExtent(new Text(tableId), null, null).toMetadataRange(),
+        true, true);
 
     if (!cloneIter.hasNext() || !srcIter.hasNext())
       throw new RuntimeException(" table deleted during clone?  srcTableId = " + srcTableId + " tableId=" + tableId);
@@ -831,12 +834,12 @@ public class MetadataTableUtil {
     while (true) {
 
       try {
-        initializeClone(srcTableId, tableId, conn, bw);
+        initializeClone(MetadataTable.NAME, srcTableId, tableId, conn, bw);
 
         // the following loop looks changes in the file that occurred during the copy.. if files were dereferenced then they could have been GCed
 
         while (true) {
-          int rewrites = checkClone(srcTableId, tableId, conn, bw);
+          int rewrites = checkClone(MetadataTable.NAME, srcTableId, tableId, conn, bw);
 
           if (rewrites == 0)
             break;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c62e5f/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
index 7bf5d2d..de853d5 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/TableLoadBalancerTest.java
@@ -24,12 +24,11 @@ import java.util.Map;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.UUID;
 
-import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.impl.KeyExtent;
@@ -42,13 +41,17 @@ import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.master.state.TabletMigration;
 import org.apache.hadoop.io.Text;
 import org.apache.thrift.TException;
+import org.easymock.EasyMock;
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.google.common.collect.ImmutableMap;
 import com.google.common.net.HostAndPort;
 
 public class TableLoadBalancerTest {
 
+  private static Map<String,String> TABLE_ID_MAP = ImmutableMap.of("t1", "a1", "t2", "b12", "t3", "c4");
+
   static private TServerInstance mkts(String address, String session) throws Exception {
     return new TServerInstance(HostAndPort.fromParts(address, 1234), session);
   }
@@ -71,8 +74,6 @@ public class TableLoadBalancerTest {
     return result;
   }
 
-  static Instance instance = new org.apache.accumulo.core.client.mock.MockInstance(TableLoadBalancerTest.class.getName());
-
   static SortedMap<TServerInstance,TabletServerStatus> state;
 
   static List<TabletStats> generateFakeTablets(TServerInstance tserver, String tableId) {
@@ -95,6 +96,9 @@ public class TableLoadBalancerTest {
     }
 
     @Override
+    public void init(ServerConfigurationFactory conf) {}
+
+    @Override
     public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, String tableId) throws ThriftSecurityException, TException {
       return generateFakeTablets(tserver, tableId);
     }
@@ -107,6 +111,9 @@ public class TableLoadBalancerTest {
       super();
     }
 
+    @Override
+    public void init(ServerConfigurationFactory conf) {}
+
     // use our new classname to test class loading
     @Override
     protected String getLoadBalancerClassNameForTable(String table) {
@@ -118,15 +125,26 @@ public class TableLoadBalancerTest {
     public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, String tableId) throws ThriftSecurityException, TException {
       return generateFakeTablets(tserver, tableId);
     }
+
+    @Override
+    protected TableOperations getTableOperations() {
+      TableOperations tops = EasyMock.createMock(TableOperations.class);
+      EasyMock.expect(tops.tableIdMap()).andReturn(TABLE_ID_MAP).anyTimes();
+      EasyMock.replay(tops);
+      return tops;
+    }
   }
 
   @Test
   public void test() throws Exception {
-    Connector c = instance.getConnector("user", new PasswordToken("pass"));
-    ServerConfigurationFactory confFactory = new ServerConfigurationFactory(instance) {
+    final Instance inst = EasyMock.createMock(Instance.class);
+    EasyMock.expect(inst.getInstanceID()).andReturn(UUID.nameUUIDFromBytes(new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 0}).toString()).anyTimes();
+    EasyMock.replay(inst);
+
+    ServerConfigurationFactory confFactory = new ServerConfigurationFactory(inst) {
       @Override
       public TableConfiguration getTableConfiguration(String tableId) {
-        return new TableConfiguration(instance, tableId, null) {
+        return new TableConfiguration(inst, tableId, null) {
           @Override
           public String get(Property property) {
             // fake the get table configuration so the test doesn't try to look in zookeeper for per-table classpath stuff
@@ -135,11 +153,8 @@ public class TableLoadBalancerTest {
         };
       }
     };
-    TableOperations tops = c.tableOperations();
-    tops.create("t1");
-    tops.create("t2");
-    tops.create("t3");
-    String t1Id = tops.tableIdMap().get("t1"), t2Id = tops.tableIdMap().get("t2"), t3Id = tops.tableIdMap().get("t3");
+
+    String t1Id = TABLE_ID_MAP.get("t1"), t2Id = TABLE_ID_MAP.get("t2"), t3Id = TABLE_ID_MAP.get("t3");
     state = new TreeMap<TServerInstance,TabletServerStatus>();
     TServerInstance svr = mkts("10.0.0.1", "0x01020304");
     state.put(svr, status(t1Id, 10, t2Id, 10, t3Id, 10));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c62e5f/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java b/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java
index 81a3892..57c68c4 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/security/SystemCredentialsTest.java
@@ -28,6 +28,7 @@ import org.apache.accumulo.core.client.impl.ConnectorImpl;
 import org.apache.accumulo.core.client.impl.Credentials;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.security.SystemCredentials.SystemToken;
+import org.easymock.EasyMock;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Rule;
@@ -59,7 +60,9 @@ public class SystemCredentialsTest {
 
   @Before
   public void setupInstance() {
-    inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
+    inst = EasyMock.createMock(Instance.class);
+    EasyMock.expect(inst.getInstanceID()).andReturn(UUID.nameUUIDFromBytes(new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 0}).toString()).anyTimes();
+    EasyMock.replay(inst);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c62e5f/server/base/src/test/java/org/apache/accumulo/server/util/CloneTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/CloneTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/CloneTest.java
deleted file mode 100644
index aa7cad4..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/util/CloneTest.java
+++ /dev/null
@@ -1,376 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.server.util;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.HashSet;
-import java.util.Map.Entry;
-
-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.Scanner;
-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.Value;
-import org.apache.accumulo.core.data.impl.KeyExtent;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.hadoop.io.Text;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-public class CloneTest {
-
-  private Connector conn;
-
-  @Rule
-  public TestName test = new TestName();
-
-  @Before
-  public void setupInstance() throws Exception {
-    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    conn = inst.getConnector("", new PasswordToken(""));
-  }
-
-  @Test
-  public void testNoFiles() throws Exception {
-    KeyExtent ke = new KeyExtent(new Text("0"), null, null);
-    Mutation mut = ke.getPrevRowUpdateMutation();
-
-    TabletsSection.ServerColumnFamily.TIME_COLUMN.put(mut, new Value("M0".getBytes()));
-    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(mut, new Value("/default_tablet".getBytes()));
-
-    BatchWriter bw1 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-
-    bw1.addMutation(mut);
-
-    bw1.close();
-
-    BatchWriter bw2 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-
-    MetadataTableUtil.initializeClone("0", "1", conn, bw2);
-
-    int rc = MetadataTableUtil.checkClone("0", "1", conn, bw2);
-
-    assertEquals(0, rc);
-
-    // scan tables metadata entries and confirm the same
-
-  }
-
-  @Test
-  public void testFilesChange() throws Exception {
-    KeyExtent ke = new KeyExtent(new Text("0"), null, null);
-    Mutation mut = ke.getPrevRowUpdateMutation();
-
-    TabletsSection.ServerColumnFamily.TIME_COLUMN.put(mut, new Value("M0".getBytes()));
-    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(mut, new Value("/default_tablet".getBytes()));
-    mut.put(DataFileColumnFamily.NAME.toString(), "/default_tablet/0_0.rf", "1,200");
-
-    BatchWriter bw1 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-
-    bw1.addMutation(mut);
-
-    bw1.flush();
-
-    BatchWriter bw2 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-
-    MetadataTableUtil.initializeClone("0", "1", conn, bw2);
-
-    Mutation mut2 = new Mutation(ke.getMetadataEntry());
-    mut2.putDelete(DataFileColumnFamily.NAME.toString(), "/default_tablet/0_0.rf");
-    mut2.put(DataFileColumnFamily.NAME.toString(), "/default_tablet/1_0.rf", "2,300");
-
-    bw1.addMutation(mut2);
-    bw1.flush();
-
-    int rc = MetadataTableUtil.checkClone("0", "1", conn, bw2);
-
-    assertEquals(1, rc);
-
-    rc = MetadataTableUtil.checkClone("0", "1", conn, bw2);
-
-    assertEquals(0, rc);
-
-    Scanner scanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-    scanner.setRange(new KeyExtent(new Text("1"), null, null).toMetadataRange());
-
-    HashSet<String> files = new HashSet<String>();
-
-    for (Entry<Key,Value> entry : scanner) {
-      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME))
-        files.add(entry.getKey().getColumnQualifier().toString());
-    }
-
-    assertEquals(1, files.size());
-    assertTrue(files.contains("../0/default_tablet/1_0.rf"));
-
-  }
-
-  // test split where files of children are the same
-  @Test
-  public void testSplit1() throws Exception {
-    BatchWriter bw1 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-
-    bw1.addMutation(createTablet("0", null, null, "/default_tablet", "/default_tablet/0_0.rf"));
-
-    bw1.flush();
-
-    BatchWriter bw2 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-
-    MetadataTableUtil.initializeClone("0", "1", conn, bw2);
-
-    bw1.addMutation(createTablet("0", "m", null, "/default_tablet", "/default_tablet/0_0.rf"));
-    bw1.addMutation(createTablet("0", null, "m", "/t-1", "/default_tablet/0_0.rf"));
-
-    bw1.flush();
-
-    int rc = MetadataTableUtil.checkClone("0", "1", conn, bw2);
-
-    assertEquals(0, rc);
-
-    Scanner scanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-    scanner.setRange(new KeyExtent(new Text("1"), null, null).toMetadataRange());
-
-    HashSet<String> files = new HashSet<String>();
-
-    int count = 0;
-    for (Entry<Key,Value> entry : scanner) {
-      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
-        files.add(entry.getKey().getColumnQualifier().toString());
-        count++;
-      }
-    }
-
-    assertEquals(1, count);
-    assertEquals(1, files.size());
-    assertTrue(files.contains("../0/default_tablet/0_0.rf"));
-  }
-
-  // test split where files of children differ... like majc and split occurred
-  @Test
-  public void testSplit2() throws Exception {
-    BatchWriter bw1 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-
-    bw1.addMutation(createTablet("0", null, null, "/default_tablet", "/default_tablet/0_0.rf"));
-
-    bw1.flush();
-
-    BatchWriter bw2 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-
-    MetadataTableUtil.initializeClone("0", "1", conn, bw2);
-
-    bw1.addMutation(createTablet("0", "m", null, "/default_tablet", "/default_tablet/1_0.rf"));
-    Mutation mut3 = createTablet("0", null, "m", "/t-1", "/default_tablet/1_0.rf");
-    mut3.putDelete(DataFileColumnFamily.NAME.toString(), "/default_tablet/0_0.rf");
-    bw1.addMutation(mut3);
-
-    bw1.flush();
-
-    int rc = MetadataTableUtil.checkClone("0", "1", conn, bw2);
-
-    assertEquals(1, rc);
-
-    rc = MetadataTableUtil.checkClone("0", "1", conn, bw2);
-
-    assertEquals(0, rc);
-
-    Scanner scanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-    scanner.setRange(new KeyExtent(new Text("1"), null, null).toMetadataRange());
-
-    HashSet<String> files = new HashSet<String>();
-
-    int count = 0;
-
-    for (Entry<Key,Value> entry : scanner) {
-      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
-        files.add(entry.getKey().getColumnQualifier().toString());
-        count++;
-      }
-    }
-
-    assertEquals(1, files.size());
-    assertEquals(2, count);
-    assertTrue(files.contains("../0/default_tablet/1_0.rf"));
-  }
-
-  private static Mutation deleteTablet(String tid, String endRow, String prevRow, String dir, String file) throws Exception {
-    KeyExtent ke = new KeyExtent(new Text(tid), endRow == null ? null : new Text(endRow), prevRow == null ? null : new Text(prevRow));
-    Mutation mut = new Mutation(ke.getMetadataEntry());
-    TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.putDelete(mut);
-    TabletsSection.ServerColumnFamily.TIME_COLUMN.putDelete(mut);
-    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.putDelete(mut);
-    mut.putDelete(DataFileColumnFamily.NAME.toString(), file);
-
-    return mut;
-  }
-
-  private static Mutation createTablet(String tid, String endRow, String prevRow, String dir, String file) throws Exception {
-    KeyExtent ke = new KeyExtent(new Text(tid), endRow == null ? null : new Text(endRow), prevRow == null ? null : new Text(prevRow));
-    Mutation mut = ke.getPrevRowUpdateMutation();
-
-    TabletsSection.ServerColumnFamily.TIME_COLUMN.put(mut, new Value("M0".getBytes()));
-    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(mut, new Value(dir.getBytes()));
-    mut.put(DataFileColumnFamily.NAME.toString(), file, "10,200");
-
-    return mut;
-  }
-
-  // test two tablets splitting into four
-  @Test
-  public void testSplit3() throws Exception {
-    BatchWriter bw1 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-
-    bw1.addMutation(createTablet("0", "m", null, "/d1", "/d1/file1"));
-    bw1.addMutation(createTablet("0", null, "m", "/d2", "/d2/file2"));
-
-    bw1.flush();
-
-    BatchWriter bw2 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-
-    MetadataTableUtil.initializeClone("0", "1", conn, bw2);
-
-    bw1.addMutation(createTablet("0", "f", null, "/d1", "/d1/file3"));
-    bw1.addMutation(createTablet("0", "m", "f", "/d3", "/d1/file1"));
-    bw1.addMutation(createTablet("0", "s", "m", "/d2", "/d2/file2"));
-    bw1.addMutation(createTablet("0", null, "s", "/d4", "/d2/file2"));
-
-    bw1.flush();
-
-    int rc = MetadataTableUtil.checkClone("0", "1", conn, bw2);
-
-    assertEquals(0, rc);
-
-    Scanner scanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-    scanner.setRange(new KeyExtent(new Text("1"), null, null).toMetadataRange());
-
-    HashSet<String> files = new HashSet<String>();
-
-    int count = 0;
-    for (Entry<Key,Value> entry : scanner) {
-      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
-        files.add(entry.getKey().getColumnQualifier().toString());
-        count++;
-      }
-    }
-
-    assertEquals(2, count);
-    assertEquals(2, files.size());
-    assertTrue(files.contains("../0/d1/file1"));
-    assertTrue(files.contains("../0/d2/file2"));
-  }
-
-  // test cloned marker
-  @Test
-  public void testClonedMarker() throws Exception {
-    BatchWriter bw1 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-
-    bw1.addMutation(createTablet("0", "m", null, "/d1", "/d1/file1"));
-    bw1.addMutation(createTablet("0", null, "m", "/d2", "/d2/file2"));
-
-    bw1.flush();
-
-    BatchWriter bw2 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-
-    MetadataTableUtil.initializeClone("0", "1", conn, bw2);
-
-    bw1.addMutation(deleteTablet("0", "m", null, "/d1", "/d1/file1"));
-    bw1.addMutation(deleteTablet("0", null, "m", "/d2", "/d2/file2"));
-
-    bw1.flush();
-
-    bw1.addMutation(createTablet("0", "f", null, "/d1", "/d1/file3"));
-    bw1.addMutation(createTablet("0", "m", "f", "/d3", "/d1/file1"));
-    bw1.addMutation(createTablet("0", "s", "m", "/d2", "/d2/file3"));
-    bw1.addMutation(createTablet("0", null, "s", "/d4", "/d4/file3"));
-
-    bw1.flush();
-
-    int rc = MetadataTableUtil.checkClone("0", "1", conn, bw2);
-
-    assertEquals(1, rc);
-
-    bw1.addMutation(deleteTablet("0", "m", "f", "/d3", "/d1/file1"));
-
-    bw1.flush();
-
-    bw1.addMutation(createTablet("0", "m", "f", "/d3", "/d1/file3"));
-
-    bw1.flush();
-
-    rc = MetadataTableUtil.checkClone("0", "1", conn, bw2);
-
-    assertEquals(0, rc);
-
-    Scanner scanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-    scanner.setRange(new KeyExtent(new Text("1"), null, null).toMetadataRange());
-
-    HashSet<String> files = new HashSet<String>();
-
-    int count = 0;
-    for (Entry<Key,Value> entry : scanner) {
-      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
-        files.add(entry.getKey().getColumnQualifier().toString());
-        count++;
-      }
-    }
-
-    assertEquals(3, count);
-    assertEquals(3, files.size());
-    assertTrue(files.contains("../0/d1/file1"));
-    assertTrue(files.contains("../0/d2/file3"));
-    assertTrue(files.contains("../0/d4/file3"));
-  }
-
-  // test two tablets splitting into four
-  @Test
-  public void testMerge() throws Exception {
-    BatchWriter bw1 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-
-    bw1.addMutation(createTablet("0", "m", null, "/d1", "/d1/file1"));
-    bw1.addMutation(createTablet("0", null, "m", "/d2", "/d2/file2"));
-
-    bw1.flush();
-
-    BatchWriter bw2 = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-
-    MetadataTableUtil.initializeClone("0", "1", conn, bw2);
-
-    bw1.addMutation(deleteTablet("0", "m", null, "/d1", "/d1/file1"));
-    Mutation mut = createTablet("0", null, null, "/d2", "/d2/file2");
-    mut.put(DataFileColumnFamily.NAME.toString(), "/d1/file1", "10,200");
-    bw1.addMutation(mut);
-
-    bw1.flush();
-
-    try {
-      MetadataTableUtil.checkClone("0", "1", conn, bw2);
-      assertTrue(false);
-    } catch (TabletIterator.TabletDeletedException tde) {}
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c62e5f/shell/src/test/java/org/apache/accumulo/shell/ShellSetInstanceTest.java
----------------------------------------------------------------------
diff --git a/shell/src/test/java/org/apache/accumulo/shell/ShellSetInstanceTest.java b/shell/src/test/java/org/apache/accumulo/shell/ShellSetInstanceTest.java
index 4d74f65..4d2645b 100644
--- a/shell/src/test/java/org/apache/accumulo/shell/ShellSetInstanceTest.java
+++ b/shell/src/test/java/org/apache/accumulo/shell/ShellSetInstanceTest.java
@@ -49,6 +49,7 @@ import org.apache.log4j.Level;
 import org.easymock.EasyMock;
 import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Before;
 import org.junit.BeforeClass;
@@ -118,17 +119,15 @@ public class ShellSetInstanceTest {
     SiteConfiguration.clearInstance();
   }
 
+  @Deprecated
   @Test
   public void testSetInstance_Fake() throws Exception {
     ShellOptionsJC opts = createMock(ShellOptionsJC.class);
     expect(opts.isFake()).andReturn(true);
     replay(opts);
-    org.apache.accumulo.core.client.mock.MockInstance theInstance = createMock(org.apache.accumulo.core.client.mock.MockInstance.class);
-    expectNew(org.apache.accumulo.core.client.mock.MockInstance.class, "fake").andReturn(theInstance);
-    replay(theInstance, org.apache.accumulo.core.client.mock.MockInstance.class);
 
     shell.setInstance(opts);
-    verify(theInstance, org.apache.accumulo.core.client.mock.MockInstance.class);
+    Assert.assertTrue(shell.getInstance() instanceof org.apache.accumulo.core.client.mock.MockInstance);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c62e5f/test/src/main/java/org/apache/accumulo/test/CloneIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/CloneIT.java b/test/src/main/java/org/apache/accumulo/test/CloneIT.java
new file mode 100644
index 0000000..e292b75
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/CloneIT.java
@@ -0,0 +1,390 @@
+/*
+ * 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.test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.HashSet;
+import java.util.Map.Entry;
+
+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.Scanner;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.impl.KeyExtent;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.accumulo.server.util.MetadataTableUtil;
+import org.apache.accumulo.server.util.TabletIterator;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class CloneIT extends AccumuloClusterHarness {
+
+  @Test
+  public void testNoFiles() throws Exception {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    conn.tableOperations().create(tableName);
+
+    KeyExtent ke = new KeyExtent(new Text("0"), null, null);
+    Mutation mut = ke.getPrevRowUpdateMutation();
+
+    TabletsSection.ServerColumnFamily.TIME_COLUMN.put(mut, new Value("M0".getBytes()));
+    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(mut, new Value("/default_tablet".getBytes()));
+
+    BatchWriter bw1 = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    bw1.addMutation(mut);
+
+    bw1.close();
+
+    BatchWriter bw2 = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    MetadataTableUtil.initializeClone(tableName, "0", "1", conn, bw2);
+
+    int rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2);
+
+    assertEquals(0, rc);
+
+    // scan tables metadata entries and confirm the same
+
+  }
+
+  @Test
+  public void testFilesChange() throws Exception {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    conn.tableOperations().create(tableName);
+
+    KeyExtent ke = new KeyExtent(new Text("0"), null, null);
+    Mutation mut = ke.getPrevRowUpdateMutation();
+
+    TabletsSection.ServerColumnFamily.TIME_COLUMN.put(mut, new Value("M0".getBytes()));
+    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(mut, new Value("/default_tablet".getBytes()));
+    mut.put(DataFileColumnFamily.NAME.toString(), "/default_tablet/0_0.rf", "1,200");
+
+    BatchWriter bw1 = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    bw1.addMutation(mut);
+
+    bw1.flush();
+
+    BatchWriter bw2 = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    MetadataTableUtil.initializeClone(tableName, "0", "1", conn, bw2);
+
+    Mutation mut2 = new Mutation(ke.getMetadataEntry());
+    mut2.putDelete(DataFileColumnFamily.NAME.toString(), "/default_tablet/0_0.rf");
+    mut2.put(DataFileColumnFamily.NAME.toString(), "/default_tablet/1_0.rf", "2,300");
+
+    bw1.addMutation(mut2);
+    bw1.flush();
+
+    int rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2);
+
+    assertEquals(1, rc);
+
+    rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2);
+
+    assertEquals(0, rc);
+
+    Scanner scanner = conn.createScanner(tableName, Authorizations.EMPTY);
+    scanner.setRange(new KeyExtent(new Text("1"), null, null).toMetadataRange());
+
+    HashSet<String> files = new HashSet<String>();
+
+    for (Entry<Key,Value> entry : scanner) {
+      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME))
+        files.add(entry.getKey().getColumnQualifier().toString());
+    }
+
+    assertEquals(1, files.size());
+    assertTrue(files.contains("../0/default_tablet/1_0.rf"));
+
+  }
+
+  // test split where files of children are the same
+  @Test
+  public void testSplit1() throws Exception {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    conn.tableOperations().create(tableName);
+
+    BatchWriter bw1 = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    bw1.addMutation(createTablet("0", null, null, "/default_tablet", "/default_tablet/0_0.rf"));
+
+    bw1.flush();
+
+    BatchWriter bw2 = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    MetadataTableUtil.initializeClone(tableName, "0", "1", conn, bw2);
+
+    bw1.addMutation(createTablet("0", "m", null, "/default_tablet", "/default_tablet/0_0.rf"));
+    bw1.addMutation(createTablet("0", null, "m", "/t-1", "/default_tablet/0_0.rf"));
+
+    bw1.flush();
+
+    int rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2);
+
+    assertEquals(0, rc);
+
+    Scanner scanner = conn.createScanner(tableName, Authorizations.EMPTY);
+    scanner.setRange(new KeyExtent(new Text("1"), null, null).toMetadataRange());
+
+    HashSet<String> files = new HashSet<String>();
+
+    int count = 0;
+    for (Entry<Key,Value> entry : scanner) {
+      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
+        files.add(entry.getKey().getColumnQualifier().toString());
+        count++;
+      }
+    }
+
+    assertEquals(1, count);
+    assertEquals(1, files.size());
+    assertTrue(files.contains("../0/default_tablet/0_0.rf"));
+  }
+
+  // test split where files of children differ... like majc and split occurred
+  @Test
+  public void testSplit2() throws Exception {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    conn.tableOperations().create(tableName);
+
+    BatchWriter bw1 = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    bw1.addMutation(createTablet("0", null, null, "/default_tablet", "/default_tablet/0_0.rf"));
+
+    bw1.flush();
+
+    BatchWriter bw2 = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    MetadataTableUtil.initializeClone(tableName, "0", "1", conn, bw2);
+
+    bw1.addMutation(createTablet("0", "m", null, "/default_tablet", "/default_tablet/1_0.rf"));
+    Mutation mut3 = createTablet("0", null, "m", "/t-1", "/default_tablet/1_0.rf");
+    mut3.putDelete(DataFileColumnFamily.NAME.toString(), "/default_tablet/0_0.rf");
+    bw1.addMutation(mut3);
+
+    bw1.flush();
+
+    int rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2);
+
+    assertEquals(1, rc);
+
+    rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2);
+
+    assertEquals(0, rc);
+
+    Scanner scanner = conn.createScanner(tableName, Authorizations.EMPTY);
+    scanner.setRange(new KeyExtent(new Text("1"), null, null).toMetadataRange());
+
+    HashSet<String> files = new HashSet<String>();
+
+    int count = 0;
+
+    for (Entry<Key,Value> entry : scanner) {
+      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
+        files.add(entry.getKey().getColumnQualifier().toString());
+        count++;
+      }
+    }
+
+    assertEquals(1, files.size());
+    assertEquals(2, count);
+    assertTrue(files.contains("../0/default_tablet/1_0.rf"));
+  }
+
+  private static Mutation deleteTablet(String tid, String endRow, String prevRow, String dir, String file) throws Exception {
+    KeyExtent ke = new KeyExtent(new Text(tid), endRow == null ? null : new Text(endRow), prevRow == null ? null : new Text(prevRow));
+    Mutation mut = new Mutation(ke.getMetadataEntry());
+    TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.putDelete(mut);
+    TabletsSection.ServerColumnFamily.TIME_COLUMN.putDelete(mut);
+    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.putDelete(mut);
+    mut.putDelete(DataFileColumnFamily.NAME.toString(), file);
+
+    return mut;
+  }
+
+  private static Mutation createTablet(String tid, String endRow, String prevRow, String dir, String file) throws Exception {
+    KeyExtent ke = new KeyExtent(new Text(tid), endRow == null ? null : new Text(endRow), prevRow == null ? null : new Text(prevRow));
+    Mutation mut = ke.getPrevRowUpdateMutation();
+
+    TabletsSection.ServerColumnFamily.TIME_COLUMN.put(mut, new Value("M0".getBytes()));
+    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(mut, new Value(dir.getBytes()));
+    mut.put(DataFileColumnFamily.NAME.toString(), file, "10,200");
+
+    return mut;
+  }
+
+  // test two tablets splitting into four
+  @Test
+  public void testSplit3() throws Exception {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    conn.tableOperations().create(tableName);
+
+    BatchWriter bw1 = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    bw1.addMutation(createTablet("0", "m", null, "/d1", "/d1/file1"));
+    bw1.addMutation(createTablet("0", null, "m", "/d2", "/d2/file2"));
+
+    bw1.flush();
+
+    BatchWriter bw2 = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    MetadataTableUtil.initializeClone(tableName, "0", "1", conn, bw2);
+
+    bw1.addMutation(createTablet("0", "f", null, "/d1", "/d1/file3"));
+    bw1.addMutation(createTablet("0", "m", "f", "/d3", "/d1/file1"));
+    bw1.addMutation(createTablet("0", "s", "m", "/d2", "/d2/file2"));
+    bw1.addMutation(createTablet("0", null, "s", "/d4", "/d2/file2"));
+
+    bw1.flush();
+
+    int rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2);
+
+    assertEquals(0, rc);
+
+    Scanner scanner = conn.createScanner(tableName, Authorizations.EMPTY);
+    scanner.setRange(new KeyExtent(new Text("1"), null, null).toMetadataRange());
+
+    HashSet<String> files = new HashSet<String>();
+
+    int count = 0;
+    for (Entry<Key,Value> entry : scanner) {
+      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
+        files.add(entry.getKey().getColumnQualifier().toString());
+        count++;
+      }
+    }
+
+    assertEquals(2, count);
+    assertEquals(2, files.size());
+    assertTrue(files.contains("../0/d1/file1"));
+    assertTrue(files.contains("../0/d2/file2"));
+  }
+
+  // test cloned marker
+  @Test
+  public void testClonedMarker() throws Exception {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    conn.tableOperations().create(tableName);
+
+    BatchWriter bw1 = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    bw1.addMutation(createTablet("0", "m", null, "/d1", "/d1/file1"));
+    bw1.addMutation(createTablet("0", null, "m", "/d2", "/d2/file2"));
+
+    bw1.flush();
+
+    BatchWriter bw2 = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    MetadataTableUtil.initializeClone(tableName, "0", "1", conn, bw2);
+
+    bw1.addMutation(deleteTablet("0", "m", null, "/d1", "/d1/file1"));
+    bw1.addMutation(deleteTablet("0", null, "m", "/d2", "/d2/file2"));
+
+    bw1.flush();
+
+    bw1.addMutation(createTablet("0", "f", null, "/d1", "/d1/file3"));
+    bw1.addMutation(createTablet("0", "m", "f", "/d3", "/d1/file1"));
+    bw1.addMutation(createTablet("0", "s", "m", "/d2", "/d2/file3"));
+    bw1.addMutation(createTablet("0", null, "s", "/d4", "/d4/file3"));
+
+    bw1.flush();
+
+    int rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2);
+
+    assertEquals(1, rc);
+
+    bw1.addMutation(deleteTablet("0", "m", "f", "/d3", "/d1/file1"));
+
+    bw1.flush();
+
+    bw1.addMutation(createTablet("0", "m", "f", "/d3", "/d1/file3"));
+
+    bw1.flush();
+
+    rc = MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2);
+
+    assertEquals(0, rc);
+
+    Scanner scanner = conn.createScanner(tableName, Authorizations.EMPTY);
+    scanner.setRange(new KeyExtent(new Text("1"), null, null).toMetadataRange());
+
+    HashSet<String> files = new HashSet<String>();
+
+    int count = 0;
+    for (Entry<Key,Value> entry : scanner) {
+      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
+        files.add(entry.getKey().getColumnQualifier().toString());
+        count++;
+      }
+    }
+
+    assertEquals(3, count);
+    assertEquals(3, files.size());
+    assertTrue(files.contains("../0/d1/file1"));
+    assertTrue(files.contains("../0/d2/file3"));
+    assertTrue(files.contains("../0/d4/file3"));
+  }
+
+  // test two tablets splitting into four
+  @Test
+  public void testMerge() throws Exception {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(1)[0];
+    conn.tableOperations().create(tableName);
+
+    BatchWriter bw1 = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    bw1.addMutation(createTablet("0", "m", null, "/d1", "/d1/file1"));
+    bw1.addMutation(createTablet("0", null, "m", "/d2", "/d2/file2"));
+
+    bw1.flush();
+
+    BatchWriter bw2 = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    MetadataTableUtil.initializeClone(tableName, "0", "1", conn, bw2);
+
+    bw1.addMutation(deleteTablet("0", "m", null, "/d1", "/d1/file1"));
+    Mutation mut = createTablet("0", null, null, "/d2", "/d2/file2");
+    mut.put(DataFileColumnFamily.NAME.toString(), "/d1/file1", "10,200");
+    bw1.addMutation(mut);
+
+    bw1.flush();
+
+    try {
+      MetadataTableUtil.checkClone(tableName, "0", "1", conn, bw2);
+      assertTrue(false);
+    } catch (TabletIterator.TabletDeletedException tde) {}
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/66c62e5f/test/src/main/java/org/apache/accumulo/test/FindMaxIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/FindMaxIT.java b/test/src/main/java/org/apache/accumulo/test/FindMaxIT.java
new file mode 100644
index 0000000..f6e68b3
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/FindMaxIT.java
@@ -0,0 +1,113 @@
+/*
+ * 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.test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.util.ArrayList;
+import java.util.Map.Entry;
+
+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.Scanner;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.harness.AccumuloClusterHarness;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class FindMaxIT extends AccumuloClusterHarness {
+
+  private static Mutation nm(byte[] row) {
+    Mutation m = new Mutation(new Text(row));
+    m.put("cf", "cq", "v");
+    return m;
+  }
+
+  private static Mutation nm(String row) {
+    Mutation m = new Mutation(row);
+    m.put("cf", "cq", "v");
+    return m;
+  }
+
+  @Test
+  public void test1() throws Exception {
+    Connector conn = getConnector();
+    String tableName = getUniqueNames(1)[0];
+
+    conn.tableOperations().create(tableName);
+
+    BatchWriter bw = conn.createBatchWriter(tableName, new BatchWriterConfig());
+
+    bw.addMutation(nm(new byte[] {0}));
+    bw.addMutation(nm(new byte[] {0, 0}));
+    bw.addMutation(nm(new byte[] {0, 1}));
+    bw.addMutation(nm(new byte[] {0, 1, 0}));
+    bw.addMutation(nm(new byte[] {1, 0}));
+    bw.addMutation(nm(new byte[] {'a', 'b', 'c'}));
+    bw.addMutation(nm(new byte[] {(byte) 0xff}));
+    bw.addMutation(nm(new byte[] {(byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff}));
+
+    for (int i = 0; i < 1000; i += 5) {
+      bw.addMutation(nm(String.format("r%05d", i)));
+    }
+
+    bw.close();
+
+    Scanner scanner = conn.createScanner(tableName, Authorizations.EMPTY);
+
+    ArrayList<Text> rows = new ArrayList<Text>();
+
+    for (Entry<Key,Value> entry : scanner) {
+      rows.add(entry.getKey().getRow());
+    }
+
+    for (int i = rows.size() - 1; i > 0; i--) {
+      Text max = conn.tableOperations().getMaxRow(tableName, Authorizations.EMPTY, null, true, rows.get(i), false);
+      assertEquals(rows.get(i - 1), max);
+
+      max = conn.tableOperations().getMaxRow(tableName, Authorizations.EMPTY, rows.get(i - 1), true, rows.get(i), false);
+      assertEquals(rows.get(i - 1), max);
+
+      max = conn.tableOperations().getMaxRow(tableName, Authorizations.EMPTY, rows.get(i - 1), false, rows.get(i), false);
+      assertNull(max);
+
+      max = conn.tableOperations().getMaxRow(tableName, Authorizations.EMPTY, null, true, rows.get(i), true);
+      assertEquals(rows.get(i), max);
+
+      max = conn.tableOperations().getMaxRow(tableName, Authorizations.EMPTY, rows.get(i), true, rows.get(i), true);
+      assertEquals(rows.get(i), max);
+
+      max = conn.tableOperations().getMaxRow(tableName, Authorizations.EMPTY, rows.get(i - 1), false, rows.get(i), true);
+      assertEquals(rows.get(i), max);
+
+    }
+
+    Text max = conn.tableOperations().getMaxRow(tableName, Authorizations.EMPTY, null, true, null, true);
+    assertEquals(rows.get(rows.size() - 1), max);
+
+    max = conn.tableOperations().getMaxRow(tableName, Authorizations.EMPTY, null, true, new Text(new byte[] {0}), false);
+    assertNull(max);
+
+    max = conn.tableOperations().getMaxRow(tableName, Authorizations.EMPTY, null, true, new Text(new byte[] {0}), true);
+    assertEquals(rows.get(0), max);
+  }
+}


[05/14] accumulo git commit: ACCUMULO-3920 Convert more tests from mock

Posted by ct...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java b/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
deleted file mode 100644
index a8fe771..0000000
--- a/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.master;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.BatchDeleter;
-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.MutationsRejectedException;
-import org.apache.accumulo.core.client.TableNotFoundException;
-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.data.impl.KeyExtent;
-import org.apache.accumulo.core.master.thrift.MasterState;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ChoppedColumnFamily;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.master.state.MergeStats;
-import org.apache.accumulo.server.AccumuloServerContext;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
-import org.apache.accumulo.server.master.state.Assignment;
-import org.apache.accumulo.server.master.state.CurrentState;
-import org.apache.accumulo.server.master.state.MergeInfo;
-import org.apache.accumulo.server.master.state.MergeState;
-import org.apache.accumulo.server.master.state.MetaDataStateStore;
-import org.apache.accumulo.server.master.state.TServerInstance;
-import org.apache.accumulo.server.master.state.TabletLocationState;
-import org.apache.hadoop.io.Text;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-import com.google.common.net.HostAndPort;
-
-public class TestMergeState {
-
-  class MockCurrentState implements CurrentState {
-
-    TServerInstance someTServer = new TServerInstance(HostAndPort.fromParts("127.0.0.1", 1234), 0x123456);
-    MergeInfo mergeInfo;
-
-    MockCurrentState(MergeInfo info) {
-      this.mergeInfo = info;
-    }
-
-    @Override
-    public Set<String> onlineTables() {
-      return Collections.singleton("t");
-    }
-
-    @Override
-    public Set<TServerInstance> onlineTabletServers() {
-      return Collections.singleton(someTServer);
-    }
-
-    @Override
-    public Collection<MergeInfo> merges() {
-      return Collections.singleton(mergeInfo);
-    }
-
-    @Override
-    public Collection<KeyExtent> migrations() {
-      return Collections.emptyList();
-    }
-
-    @Override
-    public MasterState getMasterState() {
-      return MasterState.NORMAL;
-    }
-
-    @Override
-    public Set<TServerInstance> shutdownServers() {
-      return Collections.emptySet();
-    }
-  }
-
-  private static void update(Connector c, Mutation m) throws TableNotFoundException, MutationsRejectedException {
-    BatchWriter bw = c.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    bw.addMutation(m);
-    bw.close();
-  }
-
-  @Rule
-  public TestName test = new TestName();
-
-  @Test
-  public void test() throws Exception {
-    Instance instance = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    AccumuloServerContext context = new AccumuloServerContext(new ServerConfigurationFactory(instance));
-    Connector connector = context.getConnector();
-    BatchWriter bw = connector.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-
-    // Create a fake METADATA table with these splits
-    String splits[] = {"a", "e", "j", "o", "t", "z"};
-    // create metadata for a table "t" with the splits above
-    Text tableId = new Text("t");
-    Text pr = null;
-    for (String s : splits) {
-      Text split = new Text(s);
-      Mutation prevRow = KeyExtent.getPrevRowUpdateMutation(new KeyExtent(tableId, split, pr));
-      prevRow.put(TabletsSection.CurrentLocationColumnFamily.NAME, new Text("123456"), new Value("127.0.0.1:1234".getBytes()));
-      ChoppedColumnFamily.CHOPPED_COLUMN.put(prevRow, new Value("junk".getBytes()));
-      bw.addMutation(prevRow);
-      pr = split;
-    }
-    // Add the default tablet
-    Mutation defaultTablet = KeyExtent.getPrevRowUpdateMutation(new KeyExtent(tableId, null, pr));
-    defaultTablet.put(TabletsSection.CurrentLocationColumnFamily.NAME, new Text("123456"), new Value("127.0.0.1:1234".getBytes()));
-    bw.addMutation(defaultTablet);
-    bw.close();
-
-    // Read out the TabletLocationStates
-    MockCurrentState state = new MockCurrentState(new MergeInfo(new KeyExtent(tableId, new Text("p"), new Text("e")), MergeInfo.Operation.MERGE));
-
-    // Verify the tablet state: hosted, and count
-    MetaDataStateStore metaDataStateStore = new MetaDataStateStore(context, state);
-    int count = 0;
-    for (TabletLocationState tss : metaDataStateStore) {
-      if (tss != null)
-        count++;
-    }
-    Assert.assertEquals(0, count); // the normal case is to skip tablets in a good state
-
-    // Create the hole
-    // Split the tablet at one end of the range
-    Mutation m = new KeyExtent(tableId, new Text("t"), new Text("p")).getPrevRowUpdateMutation();
-    TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.put(m, new Value("0.5".getBytes()));
-    TabletsSection.TabletColumnFamily.OLD_PREV_ROW_COLUMN.put(m, KeyExtent.encodePrevEndRow(new Text("o")));
-    update(connector, m);
-
-    // do the state check
-    MergeStats stats = scan(state, metaDataStateStore);
-    MergeState newState = stats.nextMergeState(connector, state);
-    Assert.assertEquals(MergeState.WAITING_FOR_OFFLINE, newState);
-
-    // unassign the tablets
-    BatchDeleter deleter = connector.createBatchDeleter(MetadataTable.NAME, Authorizations.EMPTY, 1000, new BatchWriterConfig());
-    deleter.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
-    deleter.setRanges(Collections.singletonList(new Range()));
-    deleter.delete();
-
-    // now we should be ready to merge but, we have inconsistent metadata
-    stats = scan(state, metaDataStateStore);
-    Assert.assertEquals(MergeState.WAITING_FOR_OFFLINE, stats.nextMergeState(connector, state));
-
-    // finish the split
-    KeyExtent tablet = new KeyExtent(tableId, new Text("p"), new Text("o"));
-    m = tablet.getPrevRowUpdateMutation();
-    TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.put(m, new Value("0.5".getBytes()));
-    update(connector, m);
-    metaDataStateStore.setLocations(Collections.singletonList(new Assignment(tablet, state.someTServer)));
-
-    // onos... there's a new tablet online
-    stats = scan(state, metaDataStateStore);
-    Assert.assertEquals(MergeState.WAITING_FOR_CHOPPED, stats.nextMergeState(connector, state));
-
-    // chop it
-    m = tablet.getPrevRowUpdateMutation();
-    ChoppedColumnFamily.CHOPPED_COLUMN.put(m, new Value("junk".getBytes()));
-    update(connector, m);
-
-    stats = scan(state, metaDataStateStore);
-    Assert.assertEquals(MergeState.WAITING_FOR_OFFLINE, stats.nextMergeState(connector, state));
-
-    // take it offline
-    m = tablet.getPrevRowUpdateMutation();
-    Collection<Collection<String>> walogs = Collections.emptyList();
-    metaDataStateStore.unassign(Collections.singletonList(new TabletLocationState(tablet, null, state.someTServer, null, walogs, false)), null);
-
-    // now we can split
-    stats = scan(state, metaDataStateStore);
-    Assert.assertEquals(MergeState.MERGING, stats.nextMergeState(connector, state));
-
-  }
-
-  private MergeStats scan(MockCurrentState state, MetaDataStateStore metaDataStateStore) {
-    MergeStats stats = new MergeStats(state.mergeInfo);
-    stats.getMergeInfo().setState(MergeState.WAITING_FOR_OFFLINE);
-    for (TabletLocationState tss : metaDataStateStore) {
-      stats.update(tss.extent, tss.getState(state.onlineTabletServers()), tss.chopped, false);
-    }
-    return stats;
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/server/master/src/test/java/org/apache/accumulo/master/replication/FinishedWorkUpdaterTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/FinishedWorkUpdaterTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/FinishedWorkUpdaterTest.java
deleted file mode 100644
index 864a79d..0000000
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/FinishedWorkUpdaterTest.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.master.replication;
-
-import java.util.Map.Entry;
-import java.util.UUID;
-
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.Scanner;
-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.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-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.Iterables;
-
-public class FinishedWorkUpdaterTest {
-
-  @Rule
-  public TestName test = new TestName();
-
-  private Connector conn;
-  private FinishedWorkUpdater updater;
-
-  @Before
-  public void setup() throws Exception {
-    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    conn = inst.getConnector("root", new PasswordToken(""));
-    updater = new FinishedWorkUpdater(conn);
-  }
-
-  @Test
-  public void offlineReplicationTableFailsGracefully() {
-    updater.run();
-  }
-
-  @Test
-  public void recordsWithProgressUpdateBothTables() throws Exception {
-    String file = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    Status stat = Status.newBuilder().setBegin(100).setEnd(200).setClosed(true).setInfiniteEnd(false).build();
-    ReplicationTarget target = new ReplicationTarget("peer", "table1", "1");
-
-    // Create a single work record for a file to some peer
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
-    Mutation m = new Mutation(file);
-    WorkSection.add(m, target.toText(), ProtobufUtil.toValue(stat));
-    bw.addMutation(m);
-    bw.close();
-
-    updater.run();
-
-    Scanner s = ReplicationTable.getScanner(conn);
-    s.setRange(Range.exact(file));
-    StatusSection.limit(s);
-    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
-
-    Assert.assertEquals(entry.getKey().getColumnFamily(), StatusSection.NAME);
-    Assert.assertEquals(entry.getKey().getColumnQualifier().toString(), target.getSourceTableId());
-
-    // We should only rely on the correct begin attribute being returned
-    Status actual = Status.parseFrom(entry.getValue().get());
-    Assert.assertEquals(stat.getBegin(), actual.getBegin());
-  }
-
-  @Test
-  public void chooseMinimumBeginOffset() throws Exception {
-    String file = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    // @formatter:off
-    Status stat1 = Status.newBuilder().setBegin(100).setEnd(1000).setClosed(true).setInfiniteEnd(false).build(),
-        stat2 = Status.newBuilder().setBegin(500).setEnd(1000).setClosed(true).setInfiniteEnd(false).build(),
-        stat3 = Status.newBuilder().setBegin(1).setEnd(1000).setClosed(true).setInfiniteEnd(false).build();
-    ReplicationTarget target1 = new ReplicationTarget("peer1", "table1", "1"),
-        target2 = new ReplicationTarget("peer2", "table2", "1"),
-        target3 = new ReplicationTarget("peer3", "table3", "1");
-    // @formatter:on
-
-    // Create a single work record for a file to some peer
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
-    Mutation m = new Mutation(file);
-    WorkSection.add(m, target1.toText(), ProtobufUtil.toValue(stat1));
-    WorkSection.add(m, target2.toText(), ProtobufUtil.toValue(stat2));
-    WorkSection.add(m, target3.toText(), ProtobufUtil.toValue(stat3));
-    bw.addMutation(m);
-    bw.close();
-
-    updater.run();
-
-    Scanner s = ReplicationTable.getScanner(conn);
-    s.setRange(Range.exact(file));
-    StatusSection.limit(s);
-    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
-
-    Assert.assertEquals(entry.getKey().getColumnFamily(), StatusSection.NAME);
-    Assert.assertEquals(entry.getKey().getColumnQualifier().toString(), target1.getSourceTableId());
-
-    // We should only rely on the correct begin attribute being returned
-    Status actual = Status.parseFrom(entry.getValue().get());
-    Assert.assertEquals(1, actual.getBegin());
-  }
-
-  @Test
-  public void chooseMinimumBeginOffsetInfiniteEnd() throws Exception {
-    String file = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
-    // @formatter:off
-    Status stat1 = Status.newBuilder().setBegin(100).setEnd(1000).setClosed(true).setInfiniteEnd(true).build(),
-        stat2 = Status.newBuilder().setBegin(1).setEnd(1000).setClosed(true).setInfiniteEnd(true).build(),
-        stat3 = Status.newBuilder().setBegin(500).setEnd(1000).setClosed(true).setInfiniteEnd(true).build();
-    ReplicationTarget target1 = new ReplicationTarget("peer1", "table1", "1"),
-        target2 = new ReplicationTarget("peer2", "table2", "1"),
-        target3 = new ReplicationTarget("peer3", "table3", "1");
-    // @formatter:on
-
-    // Create a single work record for a file to some peer
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
-    Mutation m = new Mutation(file);
-    WorkSection.add(m, target1.toText(), ProtobufUtil.toValue(stat1));
-    WorkSection.add(m, target2.toText(), ProtobufUtil.toValue(stat2));
-    WorkSection.add(m, target3.toText(), ProtobufUtil.toValue(stat3));
-    bw.addMutation(m);
-    bw.close();
-
-    updater.run();
-
-    Scanner s = ReplicationTable.getScanner(conn);
-    s.setRange(Range.exact(file));
-    StatusSection.limit(s);
-    Entry<Key,Value> entry = Iterables.getOnlyElement(s);
-
-    Assert.assertEquals(entry.getKey().getColumnFamily(), StatusSection.NAME);
-    Assert.assertEquals(entry.getKey().getColumnQualifier().toString(), target1.getSourceTableId());
-
-    // We should only rely on the correct begin attribute being returned
-    Status actual = Status.parseFrom(entry.getValue().get());
-    Assert.assertEquals(1, actual.getBegin());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java
deleted file mode 100644
index 2555077..0000000
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java
+++ /dev/null
@@ -1,327 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.master.replication;
-
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.accumulo.core.client.BatchScanner;
-import org.apache.accumulo.core.client.BatchWriter;
-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.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.user.WholeRowIterator;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.hadoop.io.Text;
-import org.easymock.EasyMock;
-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.Iterables;
-
-public class RemoveCompleteReplicationRecordsTest {
-
-  private RemoveCompleteReplicationRecords rcrr;
-  private Instance inst;
-  private Connector conn;
-
-  @Rule
-  public TestName test = new TestName();
-
-  @Before
-  public void initialize() throws Exception {
-    inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    conn = inst.getConnector("root", new PasswordToken(""));
-    rcrr = new RemoveCompleteReplicationRecords(conn);
-  }
-
-  @Test
-  public void notYetReplicationRecordsIgnored() throws Exception {
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
-    int numRecords = 3;
-    for (int i = 0; i < numRecords; i++) {
-      String file = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-      Mutation m = new Mutation(file);
-      StatusSection.add(m, new Text(Integer.toString(i)), StatusUtil.openWithUnknownLengthValue());
-      bw.addMutation(m);
-    }
-
-    bw.close();
-
-    Assert.assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(conn)));
-
-    BatchScanner bs = ReplicationTable.getBatchScanner(conn, 1);
-    bs.setRanges(Collections.singleton(new Range()));
-    IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
-    bs.addScanIterator(cfg);
-    bw = EasyMock.createMock(BatchWriter.class);
-
-    EasyMock.replay(bw);
-
-    rcrr.removeCompleteRecords(conn, bs, bw);
-    bs.close();
-
-    Assert.assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(conn)));
-  }
-
-  @Test
-  public void partiallyReplicatedRecordsIgnored() throws Exception {
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
-    int numRecords = 3;
-    Status.Builder builder = Status.newBuilder();
-    builder.setClosed(false);
-    builder.setEnd(10000);
-    builder.setInfiniteEnd(false);
-    for (int i = 0; i < numRecords; i++) {
-      String file = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-      Mutation m = new Mutation(file);
-      StatusSection.add(m, new Text(Integer.toString(i)), ProtobufUtil.toValue(builder.setBegin(1000 * (i + 1)).build()));
-      bw.addMutation(m);
-    }
-
-    bw.close();
-
-    Assert.assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(conn)));
-
-    BatchScanner bs = ReplicationTable.getBatchScanner(conn, 1);
-    bs.setRanges(Collections.singleton(new Range()));
-    IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
-    bs.addScanIterator(cfg);
-    bw = EasyMock.createMock(BatchWriter.class);
-
-    EasyMock.replay(bw);
-
-    // We don't remove any records, so we can just pass in a fake BW for both
-    rcrr.removeCompleteRecords(conn, bs, bw);
-    bs.close();
-
-    Assert.assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(conn)));
-  }
-
-  @Test
-  public void replicatedClosedWorkRecordsAreNotRemovedWithoutClosedStatusRecords() throws Exception {
-    BatchWriter replBw = ReplicationTable.getBatchWriter(conn);
-    int numRecords = 3;
-
-    Status.Builder builder = Status.newBuilder();
-    builder.setClosed(false);
-    builder.setEnd(10000);
-    builder.setInfiniteEnd(false);
-
-    // Write out numRecords entries to both replication and metadata tables, none of which are fully replicated
-    for (int i = 0; i < numRecords; i++) {
-      String file = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-      Mutation m = new Mutation(file);
-      StatusSection.add(m, new Text(Integer.toString(i)), ProtobufUtil.toValue(builder.setBegin(1000 * (i + 1)).build()));
-      replBw.addMutation(m);
-    }
-
-    // Add two records that we can delete
-    String fileToRemove = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-    Mutation m = new Mutation(fileToRemove);
-    StatusSection.add(m, new Text("5"), ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(false).build()));
-    replBw.addMutation(m);
-
-    numRecords++;
-
-    fileToRemove = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-    m = new Mutation(fileToRemove);
-    StatusSection.add(m, new Text("6"), ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(false).build()));
-    replBw.addMutation(m);
-
-    numRecords++;
-
-    replBw.flush();
-
-    // Make sure that we have the expected number of records in both tables
-    Assert.assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(conn)));
-
-    // We should not remove any records because they're missing closed status
-    BatchScanner bs = ReplicationTable.getBatchScanner(conn, 1);
-    bs.setRanges(Collections.singleton(new Range()));
-    IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
-    bs.addScanIterator(cfg);
-
-    try {
-      Assert.assertEquals(0l, rcrr.removeCompleteRecords(conn, bs, replBw));
-    } finally {
-      bs.close();
-      replBw.close();
-    }
-  }
-
-  @Test
-  public void replicatedClosedRowsAreRemoved() throws Exception {
-    BatchWriter replBw = ReplicationTable.getBatchWriter(conn);
-    int numRecords = 3;
-
-    Status.Builder builder = Status.newBuilder();
-    builder.setClosed(false);
-    builder.setEnd(10000);
-    builder.setInfiniteEnd(false);
-
-    long time = System.currentTimeMillis();
-    // Write out numRecords entries to both replication and metadata tables, none of which are fully replicated
-    for (int i = 0; i < numRecords; i++) {
-      builder.setCreatedTime(time++);
-      String file = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-      Mutation m = new Mutation(file);
-      Value v = ProtobufUtil.toValue(builder.setBegin(1000 * (i + 1)).build());
-      StatusSection.add(m, new Text(Integer.toString(i)), v);
-      replBw.addMutation(m);
-      m = OrderSection.createMutation(file, time);
-      OrderSection.add(m, new Text(Integer.toString(i)), v);
-      replBw.addMutation(m);
-    }
-
-    Set<String> filesToRemove = new HashSet<>();
-    // We created two mutations for each file
-    numRecords *= 2;
-    int finalNumRecords = numRecords;
-
-    // Add two records that we can delete
-    String fileToRemove = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-    filesToRemove.add(fileToRemove);
-    Mutation m = new Mutation(fileToRemove);
-    ReplicationTarget target = new ReplicationTarget("peer1", "5", "5");
-    Value value = ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(true).setCreatedTime(time).build());
-    StatusSection.add(m, new Text("5"), value);
-    WorkSection.add(m, target.toText(), value);
-    replBw.addMutation(m);
-
-    m = OrderSection.createMutation(fileToRemove, time);
-    OrderSection.add(m, new Text("5"), value);
-    replBw.addMutation(m);
-    time++;
-
-    numRecords += 3;
-
-    fileToRemove = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-    filesToRemove.add(fileToRemove);
-    m = new Mutation(fileToRemove);
-    value = ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(true).setCreatedTime(time).build());
-    target = new ReplicationTarget("peer1", "6", "6");
-    StatusSection.add(m, new Text("6"), value);
-    WorkSection.add(m, target.toText(), value);
-    replBw.addMutation(m);
-
-    m = OrderSection.createMutation(fileToRemove, time);
-    OrderSection.add(m, new Text("6"), value);
-    replBw.addMutation(m);
-    time++;
-
-    numRecords += 3;
-
-    replBw.flush();
-
-    // Make sure that we have the expected number of records in both tables
-    Assert.assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(conn)));
-
-    // We should remove the two fully completed records we inserted
-    BatchScanner bs = ReplicationTable.getBatchScanner(conn, 1);
-    bs.setRanges(Collections.singleton(new Range()));
-    StatusSection.limit(bs);
-    WorkSection.limit(bs);
-    IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
-    bs.addScanIterator(cfg);
-
-    try {
-      Assert.assertEquals(4l, rcrr.removeCompleteRecords(conn, bs, replBw));
-    } finally {
-      bs.close();
-      replBw.close();
-    }
-
-    int actualRecords = 0;
-    for (Entry<Key,Value> entry : ReplicationTable.getScanner(conn)) {
-      Assert.assertFalse(filesToRemove.contains(entry.getKey().getRow().toString()));
-      actualRecords++;
-    }
-
-    Assert.assertEquals(finalNumRecords, actualRecords);
-  }
-
-  @Test
-  public void partiallyReplicatedEntriesPrecludeRowDeletion() throws Exception {
-    BatchWriter replBw = ReplicationTable.getBatchWriter(conn);
-    int numRecords = 3;
-
-    Status.Builder builder = Status.newBuilder();
-    builder.setClosed(false);
-    builder.setEnd(10000);
-    builder.setInfiniteEnd(false);
-
-    // Write out numRecords entries to both replication and metadata tables, none of which are fully replicated
-    for (int i = 0; i < numRecords; i++) {
-      String file = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-      Mutation m = new Mutation(file);
-      StatusSection.add(m, new Text(Integer.toString(i)), ProtobufUtil.toValue(builder.setBegin(1000 * (i + 1)).build()));
-      replBw.addMutation(m);
-    }
-
-    // Add two records that we can delete
-    String fileToRemove = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
-    Mutation m = new Mutation(fileToRemove);
-    ReplicationTarget target = new ReplicationTarget("peer1", "5", "5");
-    Value value = ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(true).build());
-    StatusSection.add(m, new Text("5"), value);
-    WorkSection.add(m, target.toText(), value);
-    target = new ReplicationTarget("peer2", "5", "5");
-    WorkSection.add(m, target.toText(), value);
-    target = new ReplicationTarget("peer3", "5", "5");
-    WorkSection.add(m, target.toText(), ProtobufUtil.toValue(builder.setClosed(false).build()));
-    replBw.addMutation(m);
-
-    numRecords += 4;
-
-    replBw.flush();
-
-    // Make sure that we have the expected number of records in both tables
-    Assert.assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(conn)));
-
-    // We should remove the two fully completed records we inserted
-    BatchScanner bs = ReplicationTable.getBatchScanner(conn, 1);
-    bs.setRanges(Collections.singleton(new Range()));
-    IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
-    bs.addScanIterator(cfg);
-
-    try {
-      Assert.assertEquals(0l, rcrr.removeCompleteRecords(conn, bs, replBw));
-    } finally {
-      bs.close();
-      replBw.close();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
index d4675db..45fe959 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
@@ -18,260 +18,35 @@ package org.apache.accumulo.master.replication;
 
 import static org.easymock.EasyMock.createMock;
 import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.expectLastCall;
 import static org.easymock.EasyMock.replay;
 import static org.easymock.EasyMock.verify;
 
-import java.util.HashMap;
 import java.util.Map;
 import java.util.TreeMap;
 
-import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationConstants;
-import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
 import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.server.replication.DistributedWorkQueueWorkAssignerHelper;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
 import org.apache.accumulo.server.zookeeper.ZooCache;
-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;
 
 public class SequentialWorkAssignerTest {
 
-  @Rule
-  public TestName test = new TestName();
-
-  private AccumuloConfiguration conf;
   private Connector conn;
-  private Connector mockConn;
   private SequentialWorkAssigner assigner;
 
   @Before
   public void init() throws Exception {
-    conf = createMock(AccumuloConfiguration.class);
+    AccumuloConfiguration conf = createMock(AccumuloConfiguration.class);
     conn = createMock(Connector.class);
     assigner = new SequentialWorkAssigner(conf, conn);
-
-    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    mockConn = inst.getConnector("root", new PasswordToken(""));
-    // Set the connector
-    assigner.setConnector(mockConn);
-    // grant ourselves write to the replication table
-    mockConn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
-  }
-
-  @Test
-  public void createWorkForFilesInCorrectOrder() throws Exception {
-    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
-    Text serializedTarget = target.toText();
-
-    // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(mockConn);
-    // We want the name of file2 to sort before file1
-    String filename1 = "z_file1", filename2 = "a_file1";
-    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
-
-    // File1 was closed before file2, however
-    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(250).build();
-    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(500).build();
-
-    Mutation m = new Mutation(file1);
-    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = new Mutation(file2);
-    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file1, stat1.getCreatedTime());
-    OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file2, stat2.getCreatedTime());
-    OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    bw.close();
-
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-    Map<String,Map<String,String>> queuedWork = new HashMap<>();
-    assigner.setQueuedWork(queuedWork);
-    assigner.setWorkQueue(workQueue);
-    assigner.setMaxQueueSize(Integer.MAX_VALUE);
-
-    // Make sure we expect the invocations in the correct order (accumulo is sorted)
-    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target), file1);
-    expectLastCall().once();
-
-    // file2 is *not* queued because file1 must be replicated first
-
-    replay(workQueue);
-
-    assigner.createWork();
-
-    verify(workQueue);
-
-    Assert.assertEquals(1, queuedWork.size());
-    Assert.assertTrue(queuedWork.containsKey("cluster1"));
-    Map<String,String> cluster1Work = queuedWork.get("cluster1");
-    Assert.assertEquals(1, cluster1Work.size());
-    Assert.assertTrue(cluster1Work.containsKey(target.getSourceTableId()));
-    Assert.assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target), cluster1Work.get(target.getSourceTableId()));
-  }
-
-  @Test
-  public void workAcrossTablesHappensConcurrently() throws Exception {
-    ReplicationTarget target1 = new ReplicationTarget("cluster1", "table1", "1");
-    Text serializedTarget1 = target1.toText();
-
-    ReplicationTarget target2 = new ReplicationTarget("cluster1", "table2", "2");
-    Text serializedTarget2 = target2.toText();
-
-    // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(mockConn);
-    // We want the name of file2 to sort before file1
-    String filename1 = "z_file1", filename2 = "a_file1";
-    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
-
-    // File1 was closed before file2, however
-    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(250).build();
-    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(500).build();
-
-    Mutation m = new Mutation(file1);
-    WorkSection.add(m, serializedTarget1, ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = new Mutation(file2);
-    WorkSection.add(m, serializedTarget2, ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file1, stat1.getCreatedTime());
-    OrderSection.add(m, new Text(target1.getSourceTableId()), ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file2, stat2.getCreatedTime());
-    OrderSection.add(m, new Text(target2.getSourceTableId()), ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    bw.close();
-
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-    Map<String,Map<String,String>> queuedWork = new HashMap<>();
-    assigner.setQueuedWork(queuedWork);
-    assigner.setWorkQueue(workQueue);
-    assigner.setMaxQueueSize(Integer.MAX_VALUE);
-
-    // Make sure we expect the invocations in the correct order (accumulo is sorted)
-    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target1), file1);
-    expectLastCall().once();
-
-    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target2), file2);
-    expectLastCall().once();
-
-    // file2 is *not* queued because file1 must be replicated first
-
-    replay(workQueue);
-
-    assigner.createWork();
-
-    verify(workQueue);
-
-    Assert.assertEquals(1, queuedWork.size());
-    Assert.assertTrue(queuedWork.containsKey("cluster1"));
-
-    Map<String,String> cluster1Work = queuedWork.get("cluster1");
-    Assert.assertEquals(2, cluster1Work.size());
-    Assert.assertTrue(cluster1Work.containsKey(target1.getSourceTableId()));
-    Assert.assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target1), cluster1Work.get(target1.getSourceTableId()));
-
-    Assert.assertTrue(cluster1Work.containsKey(target2.getSourceTableId()));
-    Assert.assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target2), cluster1Work.get(target2.getSourceTableId()));
-  }
-
-  @Test
-  public void workAcrossPeersHappensConcurrently() throws Exception {
-    ReplicationTarget target1 = new ReplicationTarget("cluster1", "table1", "1");
-    Text serializedTarget1 = target1.toText();
-
-    ReplicationTarget target2 = new ReplicationTarget("cluster2", "table1", "1");
-    Text serializedTarget2 = target2.toText();
-
-    // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(mockConn);
-    // We want the name of file2 to sort before file1
-    String filename1 = "z_file1", filename2 = "a_file1";
-    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
-
-    // File1 was closed before file2, however
-    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(250).build();
-    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(500).build();
-
-    Mutation m = new Mutation(file1);
-    WorkSection.add(m, serializedTarget1, ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = new Mutation(file2);
-    WorkSection.add(m, serializedTarget2, ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file1, stat1.getCreatedTime());
-    OrderSection.add(m, new Text(target1.getSourceTableId()), ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file2, stat2.getCreatedTime());
-    OrderSection.add(m, new Text(target2.getSourceTableId()), ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    bw.close();
-
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-    Map<String,Map<String,String>> queuedWork = new HashMap<>();
-    assigner.setQueuedWork(queuedWork);
-    assigner.setWorkQueue(workQueue);
-    assigner.setMaxQueueSize(Integer.MAX_VALUE);
-
-    // Make sure we expect the invocations in the correct order (accumulo is sorted)
-    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target1), file1);
-    expectLastCall().once();
-
-    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target2), file2);
-    expectLastCall().once();
-
-    // file2 is *not* queued because file1 must be replicated first
-
-    replay(workQueue);
-
-    assigner.createWork();
-
-    verify(workQueue);
-
-    Assert.assertEquals(2, queuedWork.size());
-    Assert.assertTrue(queuedWork.containsKey("cluster1"));
-
-    Map<String,String> cluster1Work = queuedWork.get("cluster1");
-    Assert.assertEquals(1, cluster1Work.size());
-    Assert.assertTrue(cluster1Work.containsKey(target1.getSourceTableId()));
-    Assert.assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target1), cluster1Work.get(target1.getSourceTableId()));
-
-    Map<String,String> cluster2Work = queuedWork.get("cluster2");
-    Assert.assertEquals(1, cluster2Work.size());
-    Assert.assertTrue(cluster2Work.containsKey(target2.getSourceTableId()));
-    Assert.assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target2), cluster2Work.get(target2.getSourceTableId()));
   }
 
   @Test
@@ -315,69 +90,4 @@ public class SequentialWorkAssignerTest {
     Assert.assertEquals(1, cluster1Work.size());
     Assert.assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey("file2", new ReplicationTarget("cluster1", "2", "2")), cluster1Work.get("2"));
   }
-
-  @Test
-  public void reprocessingOfCompletedWorkRemovesWork() throws Exception {
-    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
-    Text serializedTarget = target.toText();
-
-    // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(mockConn);
-    // We want the name of file2 to sort before file1
-    String filename1 = "z_file1", filename2 = "a_file1";
-    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
-
-    // File1 was closed before file2, however
-    Status stat1 = Status.newBuilder().setBegin(100).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(250).build();
-    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(500).build();
-
-    Mutation m = new Mutation(file1);
-    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = new Mutation(file2);
-    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file1, stat1.getCreatedTime());
-    OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file2, stat2.getCreatedTime());
-    OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    bw.close();
-
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-
-    // Treat filename1 as we have already submitted it for replication
-    Map<String,Map<String,String>> queuedWork = new HashMap<>();
-    Map<String,String> queuedWorkForCluster = new HashMap<>();
-    queuedWorkForCluster.put(target.getSourceTableId(), DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename1, target));
-    queuedWork.put("cluster1", queuedWorkForCluster);
-
-    assigner.setQueuedWork(queuedWork);
-    assigner.setWorkQueue(workQueue);
-    assigner.setMaxQueueSize(Integer.MAX_VALUE);
-
-    // Make sure we expect the invocations in the correct order (accumulo is sorted)
-    workQueue.addWork(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target), file2);
-    expectLastCall().once();
-
-    // file2 is queued because we remove file1 because it's fully replicated
-
-    replay(workQueue);
-
-    assigner.createWork();
-
-    verify(workQueue);
-
-    Assert.assertEquals(1, queuedWork.size());
-    Assert.assertTrue(queuedWork.containsKey("cluster1"));
-    Map<String,String> cluster1Work = queuedWork.get("cluster1");
-    Assert.assertEquals(1, cluster1Work.size());
-    Assert.assertTrue(cluster1Work.containsKey(target.getSourceTableId()));
-    Assert.assertEquals(DistributedWorkQueueWorkAssignerHelper.getQueueKey(filename2, target), cluster1Work.get(target.getSourceTableId()));
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java
deleted file mode 100644
index 11be4fb..0000000
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java
+++ /dev/null
@@ -1,253 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.master.replication;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.UUID;
-
-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.Scanner;
-import org.apache.accumulo.core.client.impl.Credentials;
-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.Value;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.server.util.ReplicationTableUtil;
-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.Iterables;
-import com.google.common.collect.Sets;
-
-public class StatusMakerTest {
-
-  @Rule
-  public TestName test = new TestName();
-
-  private Connector conn;
-
-  @Before
-  public void setupInstance() throws Exception {
-    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    Credentials creds = new Credentials("root", new PasswordToken(""));
-    conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
-  }
-
-  @Test
-  public void statusRecordsCreated() throws Exception {
-    String sourceTable = "source";
-    conn.tableOperations().create(sourceTable);
-    ReplicationTableUtil.configureMetadataTable(conn, sourceTable);
-
-    BatchWriter bw = conn.createBatchWriter(sourceTable, new BatchWriterConfig());
-    String walPrefix = "hdfs://localhost:8020/accumulo/wals/tserver+port/";
-    Set<String> files = Sets.newHashSet(walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(),
-        walPrefix + UUID.randomUUID());
-    Map<String,Integer> fileToTableId = new HashMap<>();
-
-    int index = 1;
-    long timeCreated = 0;
-    Map<String,Long> fileToTimeCreated = new HashMap<>();
-    for (String file : files) {
-      Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
-      m.put(ReplicationSection.COLF, new Text(Integer.toString(index)), StatusUtil.fileCreatedValue(timeCreated));
-      fileToTimeCreated.put(file, timeCreated);
-      bw.addMutation(m);
-      fileToTableId.put(file, index);
-      index++;
-      timeCreated++;
-    }
-
-    bw.close();
-
-    StatusMaker statusMaker = new StatusMaker(conn);
-    statusMaker.setSourceTableName(sourceTable);
-
-    statusMaker.run();
-
-    Scanner s = ReplicationTable.getScanner(conn);
-    StatusSection.limit(s);
-    Text file = new Text(), tableId = new Text();
-    for (Entry<Key,Value> entry : s) {
-      StatusSection.getFile(entry.getKey(), file);
-      StatusSection.getTableId(entry.getKey(), tableId);
-
-      Assert.assertTrue("Found unexpected file: " + file, files.contains(file.toString()));
-      Assert.assertEquals(fileToTableId.get(file.toString()), new Integer(tableId.toString()));
-      timeCreated = fileToTimeCreated.get(file.toString());
-      Assert.assertNotNull(timeCreated);
-      Assert.assertEquals(StatusUtil.fileCreated(timeCreated), Status.parseFrom(entry.getValue().get()));
-    }
-  }
-
-  @Test
-  public void openMessagesAreNotDeleted() throws Exception {
-    String sourceTable = "source";
-    conn.tableOperations().create(sourceTable);
-    ReplicationTableUtil.configureMetadataTable(conn, sourceTable);
-
-    BatchWriter bw = conn.createBatchWriter(sourceTable, new BatchWriterConfig());
-    String walPrefix = "hdfs://localhost:8020/accumulo/wals/tserver+port/";
-    Set<String> files = Sets.newHashSet(walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(),
-        walPrefix + UUID.randomUUID());
-    Map<String,Integer> fileToTableId = new HashMap<>();
-
-    int index = 1;
-    long timeCreated = 0;
-    for (String file : files) {
-      Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
-      m.put(ReplicationSection.COLF, new Text(Integer.toString(index)), StatusUtil.fileCreatedValue(timeCreated));
-      bw.addMutation(m);
-      fileToTableId.put(file, index);
-      index++;
-      timeCreated++;
-    }
-
-    bw.close();
-
-    StatusMaker statusMaker = new StatusMaker(conn);
-    statusMaker.setSourceTableName(sourceTable);
-
-    statusMaker.run();
-
-    Scanner s = conn.createScanner(sourceTable, Authorizations.EMPTY);
-    s.setRange(ReplicationSection.getRange());
-    s.fetchColumnFamily(ReplicationSection.COLF);
-    Assert.assertEquals(files.size(), Iterables.size(s));
-  }
-
-  @Test
-  public void closedMessagesAreDeleted() throws Exception {
-    String sourceTable = "source";
-    conn.tableOperations().create(sourceTable);
-    ReplicationTableUtil.configureMetadataTable(conn, sourceTable);
-
-    BatchWriter bw = conn.createBatchWriter(sourceTable, new BatchWriterConfig());
-    String walPrefix = "hdfs://localhost:8020/accumulo/wals/tserver+port/";
-    Set<String> files = Sets.newHashSet(walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(),
-        walPrefix + UUID.randomUUID());
-    Map<String,Integer> fileToTableId = new HashMap<>();
-
-    Status stat = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(true).setCreatedTime(System.currentTimeMillis()).build();
-
-    int index = 1;
-    for (String file : files) {
-      Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
-      m.put(ReplicationSection.COLF, new Text(Integer.toString(index)), ProtobufUtil.toValue(stat));
-      bw.addMutation(m);
-      fileToTableId.put(file, index);
-      index++;
-    }
-
-    bw.close();
-
-    StatusMaker statusMaker = new StatusMaker(conn);
-    statusMaker.setSourceTableName(sourceTable);
-
-    statusMaker.run();
-
-    Scanner s = conn.createScanner(sourceTable, Authorizations.EMPTY);
-    s.setRange(ReplicationSection.getRange());
-    s.fetchColumnFamily(ReplicationSection.COLF);
-    for (Entry<Key,Value> e : s) {
-      System.out.println(e.getKey().toStringNoTruncate() + " " + e.getValue());
-    }
-    s = conn.createScanner(sourceTable, Authorizations.EMPTY);
-    s.setRange(ReplicationSection.getRange());
-    s.fetchColumnFamily(ReplicationSection.COLF);
-    Assert.assertEquals(0, Iterables.size(s));
-
-  }
-
-  @Test
-  public void closedMessagesCreateOrderRecords() throws Exception {
-    String sourceTable = "source";
-    conn.tableOperations().create(sourceTable);
-    ReplicationTableUtil.configureMetadataTable(conn, sourceTable);
-
-    BatchWriter bw = conn.createBatchWriter(sourceTable, new BatchWriterConfig());
-    String walPrefix = "hdfs://localhost:8020/accumulo/wals/tserver+port/";
-    List<String> files = Arrays.asList(walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(),
-        walPrefix + UUID.randomUUID());
-    Map<String,Integer> fileToTableId = new HashMap<>();
-
-    Status.Builder statBuilder = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(true);
-
-    int index = 1;
-    long time = System.currentTimeMillis();
-    for (String file : files) {
-      statBuilder.setCreatedTime(time++);
-      Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
-      m.put(ReplicationSection.COLF, new Text(Integer.toString(index)), ProtobufUtil.toValue(statBuilder.build()));
-      bw.addMutation(m);
-      fileToTableId.put(file, index);
-      index++;
-    }
-
-    bw.close();
-
-    StatusMaker statusMaker = new StatusMaker(conn);
-    statusMaker.setSourceTableName(sourceTable);
-
-    statusMaker.run();
-
-    Scanner s = conn.createScanner(sourceTable, Authorizations.EMPTY);
-    s.setRange(ReplicationSection.getRange());
-    s.fetchColumnFamily(ReplicationSection.COLF);
-    Assert.assertEquals(0, Iterables.size(s));
-
-    s = ReplicationTable.getScanner(conn);
-    OrderSection.limit(s);
-    Iterator<Entry<Key,Value>> iter = s.iterator();
-    Assert.assertTrue("Found no order records in replication table", iter.hasNext());
-
-    Iterator<String> expectedFiles = files.iterator();
-    Text buff = new Text();
-    while (expectedFiles.hasNext() && iter.hasNext()) {
-      String file = expectedFiles.next();
-      Entry<Key,Value> entry = iter.next();
-
-      Assert.assertEquals(file, OrderSection.getFile(entry.getKey(), buff));
-      OrderSection.getTableId(entry.getKey(), buff);
-      Assert.assertEquals(fileToTableId.get(file).intValue(), Integer.parseInt(buff.toString()));
-    }
-
-    Assert.assertFalse("Found more files unexpectedly", expectedFiles.hasNext());
-    Assert.assertFalse("Found more entries in replication table unexpectedly", iter.hasNext());
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/server/master/src/test/java/org/apache/accumulo/master/replication/UnorderedWorkAssignerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/UnorderedWorkAssignerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/UnorderedWorkAssignerTest.java
index f4c53f0..a9af68b 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/UnorderedWorkAssignerTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/UnorderedWorkAssignerTest.java
@@ -30,50 +30,29 @@ import java.util.Set;
 import java.util.UUID;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationConstants;
-import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
 import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.server.replication.DistributedWorkQueueWorkAssignerHelper;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
 import org.apache.accumulo.server.zookeeper.ZooCache;
 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;
 
 public class UnorderedWorkAssignerTest {
 
-  @Rule
-  public TestName test = new TestName();
-
-  private AccumuloConfiguration conf;
   private Connector conn;
-  private Connector mockConn;
   private UnorderedWorkAssigner assigner;
 
   @Before
   public void init() throws Exception {
-    conf = createMock(AccumuloConfiguration.class);
+    AccumuloConfiguration conf = createMock(AccumuloConfiguration.class);
     conn = createMock(Connector.class);
     assigner = new UnorderedWorkAssigner(conf, conn);
-
-    Instance inst = new org.apache.accumulo.core.client.mock.MockInstance(test.getMethodName());
-    mockConn = inst.getConnector("root", new PasswordToken(""));
   }
 
   @Test
@@ -122,106 +101,6 @@ public class UnorderedWorkAssignerTest {
   }
 
   @Test
-  public void createWorkForFilesNeedingIt() throws Exception {
-    ReplicationTarget target1 = new ReplicationTarget("cluster1", "table1", "1"), target2 = new ReplicationTarget("cluster1", "table2", "2");
-    Text serializedTarget1 = target1.toText(), serializedTarget2 = target2.toText();
-    String keyTarget1 = target1.getPeerName() + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target1.getRemoteIdentifier()
-        + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target1.getSourceTableId(), keyTarget2 = target2.getPeerName()
-        + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target2.getRemoteIdentifier() + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR
-        + target2.getSourceTableId();
-
-    // Set the connector
-    assigner.setConnector(mockConn);
-
-    // grant ourselves write to the replication table
-    mockConn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
-
-    Status.Builder builder = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(false).setCreatedTime(5l);
-    Status status1 = builder.build();
-    builder.setCreatedTime(10l);
-    Status status2 = builder.build();
-
-    // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(mockConn);
-    String filename1 = UUID.randomUUID().toString(), filename2 = UUID.randomUUID().toString();
-    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
-    Mutation m = new Mutation(file1);
-    WorkSection.add(m, serializedTarget1, ProtobufUtil.toValue(status1));
-    bw.addMutation(m);
-    m = OrderSection.createMutation(file1, status1.getCreatedTime());
-    OrderSection.add(m, new Text(target1.getSourceTableId()), ProtobufUtil.toValue(status1));
-    bw.addMutation(m);
-
-    m = new Mutation(file2);
-    WorkSection.add(m, serializedTarget2, ProtobufUtil.toValue(status2));
-    bw.addMutation(m);
-    m = OrderSection.createMutation(file2, status2.getCreatedTime());
-    OrderSection.add(m, new Text(target2.getSourceTableId()), ProtobufUtil.toValue(status2));
-    bw.addMutation(m);
-
-    bw.close();
-
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-    HashSet<String> queuedWork = new HashSet<>();
-    assigner.setQueuedWork(queuedWork);
-    assigner.setWorkQueue(workQueue);
-    assigner.setMaxQueueSize(Integer.MAX_VALUE);
-
-    // Make sure we expect the invocations in the order they were created
-    String key = filename1 + "|" + keyTarget1;
-    workQueue.addWork(key, file1);
-    expectLastCall().once();
-
-    key = filename2 + "|" + keyTarget2;
-    workQueue.addWork(key, file2);
-    expectLastCall().once();
-
-    replay(workQueue);
-
-    assigner.createWork();
-
-    verify(workQueue);
-  }
-
-  @Test
-  public void doNotCreateWorkForFilesNotNeedingIt() throws Exception {
-    ReplicationTarget target1 = new ReplicationTarget("cluster1", "table1", "1"), target2 = new ReplicationTarget("cluster1", "table2", "2");
-    Text serializedTarget1 = target1.toText(), serializedTarget2 = target2.toText();
-
-    // Set the connector
-    assigner.setConnector(mockConn);
-
-    // grant ourselves write to the replication table
-    mockConn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
-
-    // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(mockConn);
-    String filename1 = UUID.randomUUID().toString(), filename2 = UUID.randomUUID().toString();
-    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
-
-    Mutation m = new Mutation(file1);
-    WorkSection.add(m, serializedTarget1, StatusUtil.fileCreatedValue(5));
-    bw.addMutation(m);
-
-    m = new Mutation(file2);
-    WorkSection.add(m, serializedTarget2, StatusUtil.fileCreatedValue(10));
-    bw.addMutation(m);
-
-    bw.close();
-
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-    HashSet<String> queuedWork = new HashSet<>();
-    assigner.setQueuedWork(queuedWork);
-    assigner.setMaxQueueSize(Integer.MAX_VALUE);
-
-    replay(workQueue);
-
-    assigner.createWork();
-
-    verify(workQueue);
-  }
-
-  @Test
   public void workNotInZooKeeperIsCleanedUp() {
     Set<String> queuedWork = new LinkedHashSet<>(Arrays.asList("wal1", "wal2"));
     assigner.setQueuedWork(queuedWork);
@@ -243,41 +122,4 @@ public class UnorderedWorkAssignerTest {
     Assert.assertTrue("Queued work was not emptied", queuedWork.isEmpty());
   }
 
-  @Test
-  public void workNotReAdded() throws Exception {
-    Set<String> queuedWork = new HashSet<>();
-
-    assigner.setQueuedWork(queuedWork);
-
-    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
-    String serializedTarget = target.getPeerName() + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target.getRemoteIdentifier()
-        + DistributedWorkQueueWorkAssignerHelper.KEY_SEPARATOR + target.getSourceTableId();
-
-    queuedWork.add("wal1|" + serializedTarget.toString());
-
-    // Set the connector
-    assigner.setConnector(mockConn);
-
-    // grant ourselves write to the replication table
-    mockConn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
-
-    // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(mockConn);
-    String file1 = "/accumulo/wal/tserver+port/wal1";
-    Mutation m = new Mutation(file1);
-    WorkSection.add(m, target.toText(), StatusUtil.openWithUnknownLengthValue());
-    bw.addMutation(m);
-
-    bw.close();
-
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-    assigner.setWorkQueue(workQueue);
-    assigner.setMaxQueueSize(Integer.MAX_VALUE);
-
-    replay(workQueue);
-
-    assigner.createWork();
-
-    verify(workQueue);
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java
index d6d0b9a..ec849f4 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java
@@ -16,185 +16,16 @@
  */
 package org.apache.accumulo.master.replication;
 
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.accumulo.core.client.BatchWriter;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.Scanner;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.server.AccumuloServerContext;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.replication.StatusUtil;
 import org.apache.accumulo.server.replication.proto.Replication.Status;
-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.ImmutableMap;
-import com.google.common.collect.Iterables;
 
 public class WorkMakerTest {
 
-  private Instance instance;
-  private Connector conn;
-
-  @Rule
-  public TestName name = new TestName();
-  private AccumuloServerContext context;
-
-  @Before
-  public void createMockAccumulo() throws Exception {
-    instance = new org.apache.accumulo.core.client.mock.MockInstance(name.getMethodName());
-    context = new AccumuloServerContext(new ServerConfigurationFactory(instance));
-    conn = context.getConnector();
-    conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
-    conn.tableOperations().deleteRows(ReplicationTable.NAME, null, null);
-  }
-
-  @Test
-  public void singleUnitSingleTarget() throws Exception {
-    String table = name.getMethodName();
-    conn.tableOperations().create(name.getMethodName());
-    String tableId = conn.tableOperations().tableIdMap().get(table);
-    String file = "hdfs://localhost:8020/accumulo/wal/123456-1234-1234-12345678";
-
-    // Create a status record for a file
-    long timeCreated = System.currentTimeMillis();
-    Mutation m = new Mutation(new Path(file).toString());
-    m.put(StatusSection.NAME, new Text(tableId), StatusUtil.fileCreatedValue(timeCreated));
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
-    bw.addMutation(m);
-    bw.flush();
-
-    // Assert that we have one record in the status section
-    Scanner s = ReplicationTable.getScanner(conn);
-    StatusSection.limit(s);
-    Assert.assertEquals(1, Iterables.size(s));
-
-    WorkMaker workMaker = new WorkMaker(context, conn);
-
-    // Invoke the addWorkRecord method to create a Work record from the Status record earlier
-    ReplicationTarget expected = new ReplicationTarget("remote_cluster_1", "4", tableId);
-    workMaker.setBatchWriter(bw);
-    workMaker.addWorkRecord(new Text(file), StatusUtil.fileCreatedValue(timeCreated), ImmutableMap.of("remote_cluster_1", "4"), tableId);
-
-    // Scan over just the WorkSection
-    s = ReplicationTable.getScanner(conn);
-    WorkSection.limit(s);
-
-    Entry<Key,Value> workEntry = Iterables.getOnlyElement(s);
-    Key workKey = workEntry.getKey();
-    ReplicationTarget actual = ReplicationTarget.from(workKey.getColumnQualifier());
-
-    Assert.assertEquals(file, workKey.getRow().toString());
-    Assert.assertEquals(WorkSection.NAME, workKey.getColumnFamily());
-    Assert.assertEquals(expected, actual);
-    Assert.assertEquals(workEntry.getValue(), StatusUtil.fileCreatedValue(timeCreated));
-  }
-
-  @Test
-  public void singleUnitMultipleTargets() throws Exception {
-    String table = name.getMethodName();
-    conn.tableOperations().create(name.getMethodName());
-
-    String tableId = conn.tableOperations().tableIdMap().get(table);
-
-    String file = "hdfs://localhost:8020/accumulo/wal/123456-1234-1234-12345678";
-
-    Mutation m = new Mutation(new Path(file).toString());
-    m.put(StatusSection.NAME, new Text(tableId), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
-    BatchWriter bw = ReplicationTable.getBatchWriter(context.getConnector());
-    bw.addMutation(m);
-    bw.flush();
-
-    // Assert that we have one record in the status section
-    Scanner s = ReplicationTable.getScanner(conn);
-    StatusSection.limit(s);
-    Assert.assertEquals(1, Iterables.size(s));
-
-    WorkMaker workMaker = new WorkMaker(context, conn);
-
-    Map<String,String> targetClusters = ImmutableMap.of("remote_cluster_1", "4", "remote_cluster_2", "6", "remote_cluster_3", "8");
-    Set<ReplicationTarget> expectedTargets = new HashSet<>();
-    for (Entry<String,String> cluster : targetClusters.entrySet()) {
-      expectedTargets.add(new ReplicationTarget(cluster.getKey(), cluster.getValue(), tableId));
-    }
-    workMaker.setBatchWriter(bw);
-    workMaker.addWorkRecord(new Text(file), StatusUtil.fileCreatedValue(System.currentTimeMillis()), targetClusters, tableId);
-
-    s = ReplicationTable.getScanner(conn);
-    WorkSection.limit(s);
-
-    Set<ReplicationTarget> actualTargets = new HashSet<>();
-    for (Entry<Key,Value> entry : s) {
-      Assert.assertEquals(file, entry.getKey().getRow().toString());
-      Assert.assertEquals(WorkSection.NAME, entry.getKey().getColumnFamily());
-
-      ReplicationTarget target = ReplicationTarget.from(entry.getKey().getColumnQualifier());
-      actualTargets.add(target);
-    }
-
-    for (ReplicationTarget expected : expectedTargets) {
-      Assert.assertTrue("Did not find expected target: " + expected, actualTargets.contains(expected));
-      actualTargets.remove(expected);
-    }
-
-    Assert.assertTrue("Found extra replication work entries: " + actualTargets, actualTargets.isEmpty());
-  }
-
-  @Test
-  public void dontCreateWorkForEntriesWithNothingToReplicate() throws Exception {
-    String table = name.getMethodName();
-    conn.tableOperations().create(name.getMethodName());
-    String tableId = conn.tableOperations().tableIdMap().get(table);
-    String file = "hdfs://localhost:8020/accumulo/wal/123456-1234-1234-12345678";
-
-    Mutation m = new Mutation(new Path(file).toString());
-    m.put(StatusSection.NAME, new Text(tableId), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
-    bw.addMutation(m);
-    bw.flush();
-
-    // Assert that we have one record in the status section
-    Scanner s = ReplicationTable.getScanner(conn);
-    StatusSection.limit(s);
-    Assert.assertEquals(1, Iterables.size(s));
-
-    WorkMaker workMaker = new WorkMaker(context, conn);
-
-    conn.tableOperations().setProperty(ReplicationTable.NAME, Property.TABLE_REPLICATION_TARGET.getKey() + "remote_cluster_1", "4");
-
-    workMaker.setBatchWriter(bw);
-
-    // If we don't shortcircuit out, we should get an exception because ServerConfiguration.getTableConfiguration
-    // won't work with MockAccumulo
-    workMaker.run();
-
-    s = ReplicationTable.getScanner(conn);
-    WorkSection.limit(s);
-
-    Assert.assertEquals(0, Iterables.size(s));
-  }
-
   @Test
   public void closedStatusRecordsStillMakeWork() throws Exception {
-    WorkMaker workMaker = new WorkMaker(context, conn);
+    WorkMaker workMaker = new WorkMaker(null, null);
 
     Assert.assertFalse(workMaker.shouldCreateWork(StatusUtil.fileCreated(System.currentTimeMillis())));
     Assert.assertTrue(workMaker.shouldCreateWork(StatusUtil.ingestedUntil(1000)));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/shell/src/test/java/org/apache/accumulo/shell/commands/FormatterCommandTest.java
----------------------------------------------------------------------
diff --git a/shell/src/test/java/org/apache/accumulo/shell/commands/FormatterCommandTest.java b/shell/src/test/java/org/apache/accumulo/shell/commands/FormatterCommandTest.java
deleted file mode 100644
index 02a9ee2..0000000
--- a/shell/src/test/java/org/apache/accumulo/shell/commands/FormatterCommandTest.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.shell.commands;
-
-import static org.junit.Assert.assertTrue;
-
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.file.Files;
-import java.util.Iterator;
-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.TableExistsException;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.util.format.Formatter;
-import org.apache.accumulo.shell.Shell;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Tests the shell output with Formatters
- */
-public class FormatterCommandTest {
-  ByteArrayOutputStream out = null;
-  InputStream in = null;
-
-  @Test
-  public void test() throws IOException, AccumuloException, AccumuloSecurityException, TableExistsException, ClassNotFoundException {
-    // Keep the Shell AUDIT log off the test output
-    Logger.getLogger(Shell.class).setLevel(Level.WARN);
-
-    File config = Files.createTempFile(null, null).toFile();
-    config.deleteOnExit();
-    final String[] args = new String[] {"--config-file", config.toString(), "--fake", "-u", "root", "-p", ""};
-
-    final String[] commands = createCommands();
-
-    in = org.apache.accumulo.shell.mock.MockShell.makeCommands(commands);
-    out = new ByteArrayOutputStream();
-
-    final Shell shell = new org.apache.accumulo.shell.mock.MockShell(in, out);
-    assertTrue("Failed to configure shell without error", shell.config(args));
-
-    // Can't call createtable in the shell with MockAccumulo
-    shell.getConnector().tableOperations().create("test");
-
-    try {
-      shell.start();
-    } catch (Exception e) {
-      Assert.fail("Exception while running commands: " + e.getMessage());
-    }
-
-    shell.getReader().flush();
-
-    final String[] output = new String(out.toByteArray()).split("\n\r");
-
-    boolean formatterOn = false;
-
-    final String[] expectedDefault = new String[] {"row cf:cq []    1234abcd", "row cf1:cq1 []    9876fedc", "row2 cf:cq []    13579bdf",
-        "row2 cf1:cq []    2468ace"};
-
-    final String[] expectedFormatted = new String[] {"row cf:cq []    0x31 0x32 0x33 0x34 0x61 0x62 0x63 0x64",
-        "row cf1:cq1 []    0x39 0x38 0x37 0x36 0x66 0x65 0x64 0x63", "row2 cf:cq []    0x31 0x33 0x35 0x37 0x39 0x62 0x64 0x66",
-        "row2 cf1:cq []    0x32 0x34 0x36 0x38 0x61 0x63 0x65"};
-
-    int outputIndex = 0;
-    while (outputIndex < output.length) {
-      final String line = output[outputIndex];
-
-      if (line.startsWith("root@mock-instance")) {
-        if (line.contains("formatter")) {
-          formatterOn = true;
-        }
-
-        outputIndex++;
-      } else if (line.startsWith("row")) {
-        int expectedIndex = 0;
-        String[] comparisonData;
-
-        // Pick the type of data we expect (formatted or default)
-        if (formatterOn) {
-          comparisonData = expectedFormatted;
-        } else {
-          comparisonData = expectedDefault;
-        }
-
-        // Ensure each output is what we expected
-        while (expectedIndex + outputIndex < output.length && expectedIndex < expectedFormatted.length) {
-          Assert.assertEquals(comparisonData[expectedIndex].trim(), output[expectedIndex + outputIndex].trim());
-          expectedIndex++;
-        }
-
-        outputIndex += expectedIndex;
-      }
-    }
-  }
-
-  private String[] createCommands() {
-    return new String[] {"table test", "insert row cf cq 1234abcd", "insert row cf1 cq1 9876fedc", "insert row2 cf cq 13579bdf", "insert row2 cf1 cq 2468ace",
-        "scan", "formatter -t test -f org.apache.accumulo.core.util.shell.command.FormatterCommandTest$HexFormatter", "scan"};
-  }
-
-  /**
-   * <p>
-   * Simple <code>Formatter</code> that will convert each character in the Value from decimal to hexadecimal. Will automatically skip over characters in the
-   * value which do not fall within the [0-9,a-f] range.
-   * </p>
-   *
-   * <p>
-   * Example: <code>'0'</code> will be displayed as <code>'0x30'</code>
-   * </p>
-   */
-  public static class HexFormatter implements Formatter {
-    private Iterator<Entry<Key,Value>> iter = null;
-    private boolean printTs = false;
-
-    private final static String tab = "\t";
-    private final static String newline = "\n";
-
-    public HexFormatter() {}
-
-    @Override
-    public boolean hasNext() {
-      return this.iter.hasNext();
-    }
-
-    @Override
-    public String next() {
-      final Entry<Key,Value> entry = iter.next();
-
-      String key;
-
-      // Observe the timestamps
-      if (printTs) {
-        key = entry.getKey().toString();
-      } else {
-        key = entry.getKey().toStringNoTime();
-      }
-
-      final Value v = entry.getValue();
-
-      // Approximate how much space we'll need
-      final StringBuilder sb = new StringBuilder(key.length() + v.getSize() * 5);
-
-      sb.append(key).append(tab);
-
-      for (byte b : v.get()) {
-        if ((b >= 48 && b <= 57) || (b >= 97 && b <= 102)) {
-          sb.append(String.format("0x%x ", Integer.valueOf(b)));
-        }
-      }
-
-      sb.append(newline);
-
-      return sb.toString();
-    }
-
-    @Override
-    public void remove() {}
-
-    @Override
-    public void initialize(final Iterable<Entry<Key,Value>> scanner, final boolean printTimestamps) {
-      this.iter = scanner.iterator();
-      this.printTs = printTimestamps;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cc3c0111/test/src/main/java/org/apache/accumulo/test/AccumuloOutputFormatIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/AccumuloOutputFormatIT.java b/test/src/main/java/org/apache/accumulo/test/AccumuloOutputFormatIT.java
deleted file mode 100644
index 6b5e6eb..0000000
--- a/test/src/main/java/org/apache/accumulo/test/AccumuloOutputFormatIT.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.test;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.accumulo.core.client.BatchWriterConfig;
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.client.MutationsRejectedException;
-import org.apache.accumulo.core.client.mapred.AccumuloOutputFormat;
-import org.apache.accumulo.core.client.security.tokens.PasswordToken;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.test.functional.ConfigurableMacBase;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordWriter;
-import org.junit.Test;
-
-/**
- * Prevent regression of ACCUMULO-3709.
- */
-public class AccumuloOutputFormatIT extends ConfigurableMacBase {
-
-  private static final String TABLE = "abc";
-
-  @Override
-  protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setProperty(Property.TSERV_SESSION_MAXIDLE, "1");
-    cfg.setNumTservers(1);
-  }
-
-  @Test
-  public void testMapred() throws Exception {
-    Connector connector = getConnector();
-    // create a table and put some data in it
-    connector.tableOperations().create(TABLE);
-
-    JobConf job = new JobConf();
-    BatchWriterConfig batchConfig = new BatchWriterConfig();
-    // no flushes!!!!!
-    batchConfig.setMaxLatency(0, TimeUnit.MILLISECONDS);
-    // use a single thread to ensure our update session times out
-    batchConfig.setMaxWriteThreads(1);
-    // set the max memory so that we ensure we don't flush on the write.
-    batchConfig.setMaxMemory(Long.MAX_VALUE);
-    AccumuloOutputFormat outputFormat = new AccumuloOutputFormat();
-    AccumuloOutputFormat.setBatchWriterOptions(job, batchConfig);
-    AccumuloOutputFormat.setZooKeeperInstance(job, cluster.getClientConfig());
-    AccumuloOutputFormat.setConnectorInfo(job, "root", new PasswordToken(ROOT_PASSWORD));
-    RecordWriter<Text,Mutation> writer = outputFormat.getRecordWriter(null, job, "Test", null);
-
-    try {
-      for (int i = 0; i < 3; i++) {
-        Mutation m = new Mutation(new Text(String.format("%08d", i)));
-        for (int j = 0; j < 3; j++) {
-          m.put(new Text("cf1"), new Text("cq" + j), new Value((i + "_" + j).getBytes(UTF_8)));
-        }
-        writer.write(new Text(TABLE), m);
-      }
-
-    } catch (Exception e) {
-      e.printStackTrace();
-      // we don't want the exception to come from write
-    }
-
-    connector.securityOperations().revokeTablePermission("root", TABLE, TablePermission.WRITE);
-
-    try {
-      writer.close(null);
-      fail("Did not throw exception");
-    } catch (IOException ex) {
-      log.info(ex.getMessage(), ex);
-      assertTrue(ex.getCause() instanceof MutationsRejectedException);
-    }
-  }
-}


[13/14] accumulo git commit: Merge branch 'ACCUMULO-3920'

Posted by ct...@apache.org.
Merge branch 'ACCUMULO-3920'

This closes #40


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

Branch: refs/heads/master
Commit: da8d8a8dcb8a4803bcec56e4e9a8a28dcf5cc874
Parents: 6503223 cc3c011
Author: Christopher Tubbs <ct...@apache.org>
Authored: Thu Jul 30 17:39:18 2015 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Thu Jul 30 17:39:18 2015 -0400

----------------------------------------------------------------------
 core/src/main/findbugs/exclude-filter.xml       |   1 +
 .../apache/accumulo/core/cli/ClientOpts.java    |   4 +-
 .../core/client/ClientSideIteratorScanner.java  |   2 +-
 .../core/client/mapred/AbstractInputFormat.java |  12 +-
 .../core/client/mapred/AccumuloInputFormat.java |   2 +-
 .../mapred/AccumuloMultiTableInputFormat.java   |   2 +-
 .../client/mapred/AccumuloOutputFormat.java     |   8 +-
 .../client/mapred/AccumuloRowInputFormat.java   |   2 +-
 .../client/mapreduce/AbstractInputFormat.java   |  12 +-
 .../client/mapreduce/AccumuloInputFormat.java   |   2 +-
 .../AccumuloMultiTableInputFormat.java          |   2 +-
 .../client/mapreduce/AccumuloOutputFormat.java  |   7 +-
 .../mapreduce/AccumuloRowInputFormat.java       |   2 +-
 .../core/client/mapreduce/RangeInputSplit.java  |  16 +-
 .../core/client/mapreduce/impl/SplitUtils.java  |   4 +-
 .../mapreduce/lib/impl/ConfiguratorBase.java    |  11 +-
 .../mapreduce/lib/impl/InputConfigurator.java   |   4 +-
 .../mapreduce/lib/util/ConfiguratorBase.java    |   6 +-
 .../core/client/mock/IteratorAdapter.java       |  36 +-
 .../accumulo/core/client/mock/MockAccumulo.java |  10 +-
 .../core/client/mock/MockBatchDeleter.java      |   3 +
 .../core/client/mock/MockBatchScanner.java      |   4 +
 .../core/client/mock/MockBatchWriter.java       |   4 +
 .../core/client/mock/MockConfiguration.java     |   4 +
 .../core/client/mock/MockConnector.java         |   4 +
 .../accumulo/core/client/mock/MockInstance.java |   3 +-
 .../client/mock/MockInstanceOperations.java     |   4 +
 .../client/mock/MockMultiTableBatchWriter.java  |   4 +
 .../core/client/mock/MockNamespace.java         |   4 +
 .../client/mock/MockNamespaceOperations.java    |   4 +
 .../accumulo/core/client/mock/MockScanner.java  |   4 +
 .../core/client/mock/MockScannerBase.java       |   4 +
 .../client/mock/MockSecurityOperations.java     |   4 +
 .../accumulo/core/client/mock/MockTable.java    |   4 +
 .../core/client/mock/MockTableOperations.java   |   4 +
 .../accumulo/core/client/mock/MockUser.java     |   4 +
 .../client/mock/impl/MockTabletLocator.java     |   4 +
 .../accumulo/core/client/mock/package-info.java |  25 +
 .../core/iterators/IteratorAdapter.java         |  56 +++
 .../core/replication/ReplicationTable.java      |   4 +-
 .../accumulo/core/util/DeprecationUtil.java     |  61 +++
 .../core/client/ClientSideIteratorTest.java     | 131 -----
 .../accumulo/core/client/admin/FindMaxTest.java | 111 -----
 .../core/client/impl/ScannerImplTest.java       |  10 +-
 .../impl/TabletServerBatchReaderTest.java       |  14 +-
 .../mapred/AccumuloFileOutputFormatTest.java    | 183 -------
 .../client/mapred/AccumuloInputFormatTest.java  | 178 +------
 .../AccumuloMultiTableInputFormatTest.java      | 145 +-----
 .../client/mapred/AccumuloOutputFormatTest.java | 140 +-----
 .../mapred/AccumuloRowInputFormatTest.java      | 209 --------
 .../core/client/mapred/RangeInputSplitTest.java |   5 +-
 .../core/client/mapred/TokenFileTest.java       | 178 -------
 .../mapreduce/AccumuloFileOutputFormatTest.java | 171 -------
 .../mapreduce/AccumuloInputFormatTest.java      | 242 ----------
 .../AccumuloMultiTableInputFormatTest.java      | 143 +-----
 .../mapreduce/AccumuloOutputFormatTest.java     | 127 -----
 .../mapreduce/AccumuloRowInputFormatTest.java   | 204 --------
 .../BadPasswordSplitsAccumuloInputFormat.java   |  42 --
 .../EmptySplitsAccumuloInputFormat.java         |  45 --
 .../client/mapreduce/RangeInputSplitTest.java   |   5 +-
 .../core/client/mapreduce/TokenFileTest.java    | 171 -------
 .../mapreduce/impl/BatchInputSplitTest.java     |   6 +-
 .../lib/impl/ConfiguratorBaseTest.java          |   7 +-
 .../core/client/mock/MockConnectorTest.java     |   1 +
 .../core/client/mock/MockNamespacesTest.java    | 150 +++---
 .../client/mock/MockTableOperationsTest.java    |  57 ++-
 .../core/client/mock/TestBatchScanner821.java   |  12 +
 .../core/client/mock/TransformIterator.java     |  30 --
 .../user/IntersectingIteratorTest.java          |  67 +--
 .../core/iterators/user/RegExFilterTest.java    |  46 +-
 .../core/iterators/user/RowFilterTest.java      | 119 ++---
 .../user/TransformingIteratorTest.java          | 203 ++++----
 .../core/metadata/MetadataServicerTest.java     |  54 ++-
 .../accumulo/core/security/CredentialsTest.java |  29 +-
 .../examples/simple/dirlist/FileCount.java      |   2 +-
 .../examples/simple/dirlist/CountTest.java      |  99 ----
 .../simple/filedata/ChunkInputFormatTest.java   | 304 ------------
 .../simple/filedata/ChunkInputStreamTest.java   |  90 +---
 .../minicluster/MiniAccumuloCluster.java        |   2 +-
 .../impl/MiniAccumuloClusterImpl.java           |   5 +-
 .../org/apache/accumulo/proxy/ProxyServer.java  |   4 +-
 .../accumulo/server/AccumuloServerContext.java  |   4 +-
 .../server/cli/ClientOnDefaultTable.java        |   4 +-
 .../server/cli/ClientOnRequiredTable.java       |   4 +-
 .../apache/accumulo/server/cli/ClientOpts.java  |   4 +-
 .../accumulo/server/util/MetadataTableUtil.java |  29 +-
 .../server/AccumuloServerContextTest.java       |   4 +-
 .../server/client/BulkImporterTest.java         |  10 +-
 .../master/balancer/TableLoadBalancerTest.java  |  41 +-
 .../problems/ProblemReportingIteratorTest.java  |  16 +-
 .../server/security/SystemCredentialsTest.java  |  22 +-
 .../apache/accumulo/server/util/CloneTest.java  | 375 ---------------
 .../server/util/ReplicationTableUtilTest.java   |   6 +-
 .../server/util/TabletIteratorTest.java         | 107 -----
 .../CloseWriteAheadLogReferencesTest.java       | 175 -------
 .../master/MasterClientServiceHandler.java      |   7 +-
 .../master/ReplicationOperationsImplTest.java   | 452 -----------------
 .../apache/accumulo/master/TestMergeState.java  | 205 --------
 .../replication/FinishedWorkUpdaterTest.java    | 165 -------
 .../RemoveCompleteReplicationRecordsTest.java   | 327 -------------
 .../replication/SequentialWorkAssignerTest.java | 328 +------------
 .../master/replication/StatusMakerTest.java     | 259 ----------
 .../replication/UnorderedWorkAssignerTest.java  | 172 +------
 .../master/replication/WorkMakerTest.java       | 171 +------
 .../tserver/LargestFirstMemoryManagerTest.java  |  14 +-
 .../java/org/apache/accumulo/shell/Shell.java   |   6 +-
 .../apache/accumulo/shell/mock/MockShell.java   |   6 +-
 .../accumulo/shell/ShellSetInstanceTest.java    |   8 +-
 .../shell/commands/FormatterCommandTest.java    | 190 --------
 .../accumulo/test/AccumuloOutputFormatIT.java   | 100 ----
 .../accumulo/test/ClientSideIteratorIT.java     | 144 ++++++
 .../java/org/apache/accumulo/test/CloneIT.java  | 390 +++++++++++++++
 .../org/apache/accumulo/test/FindMaxIT.java     | 113 +++++
 .../org/apache/accumulo/test/ShellServerIT.java | 112 +++++
 .../test/examples/simple/dirlist/CountIT.java   | 101 ++++
 .../simple/filedata/ChunkInputFormatIT.java     | 319 ++++++++++++
 .../simple/filedata/ChunkInputStreamIT.java     | 174 +++++++
 .../test/functional/AccumuloInputFormatIT.java  | 212 --------
 .../accumulo/test/functional/MergeIT.java       |  78 +++
 .../CloseWriteAheadLogReferencesIT.java         | 184 +++++++
 .../test/mapred/AccumuloFileOutputFormatIT.java | 204 ++++++++
 .../test/mapred/AccumuloInputFormatIT.java      | 195 ++++++++
 .../mapred/AccumuloMultiTableInputFormatIT.java | 163 +++++++
 .../test/mapred/AccumuloOutputFormatIT.java     | 227 +++++++++
 .../test/mapred/AccumuloRowInputFormatIT.java   | 202 ++++++++
 .../accumulo/test/mapred/TokenFileIT.java       | 174 +++++++
 .../mapreduce/AccumuloFileOutputFormatIT.java   | 211 ++++++++
 .../test/mapreduce/AccumuloInputFormatIT.java   | 479 +++++++++++++++++++
 .../AccumuloMultiTableInputFormatIT.java        | 161 +++++++
 .../test/mapreduce/AccumuloOutputFormatIT.java  | 153 ++++++
 .../mapreduce/AccumuloRowInputFormatIT.java     | 197 ++++++++
 .../accumulo/test/mapreduce/TokenFileIT.java    | 167 +++++++
 .../accumulo/test/master/MergeStateIT.java      | 204 ++++++++
 .../test/replication/FinishedWorkUpdaterIT.java | 173 +++++++
 .../RemoveCompleteReplicationRecordsIT.java     | 337 +++++++++++++
 .../ReplicationOperationsImplIT.java            | 447 +++++++++++++++++
 .../replication/SequentialWorkAssignerIT.java   | 368 ++++++++++++++
 .../test/replication/StatusMakerIT.java         | 249 ++++++++++
 .../replication/UnorderedWorkAssignerIT.java    | 238 +++++++++
 .../accumulo/test/replication/WorkMakerIT.java  | 209 ++++++++
 .../accumulo/test/iterator/RegExTest.java       | 103 ++--
 141 files changed, 7233 insertions(+), 6764 deletions(-)
----------------------------------------------------------------------