You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by bh...@apache.org on 2014/03/28 20:09:36 UTC

[2/3] git commit: ACCUMULO-2470 Unit tests for server/base module

ACCUMULO-2470 Unit tests for server/base module

This commit adds unit tests to code under the server/base module. It also includes changes
to o.a.a.server.problems.ProblemReport to enable comprehensive testing.


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

Branch: refs/heads/master
Commit: d2089c898c64ce2ac6588b9052505dc4c49d971f
Parents: e5070d0
Author: Bill Havanki <bh...@cloudera.com>
Authored: Thu Mar 13 16:34:55 2014 -0400
Committer: Bill Havanki <bh...@cloudera.com>
Committed: Fri Mar 28 14:52:39 2014 -0400

----------------------------------------------------------------------
 .../accumulo/server/problems/ProblemReport.java |  33 +++-
 .../apache/accumulo/server/AccumuloTest.java    |  85 +++++++++
 .../apache/accumulo/server/ServerOptsTest.java  |  42 +++++
 .../server/conf/TableConfigurationTest.java     |  46 +++++
 .../server/master/state/MergeInfoTest.java      | 166 +++++++++++++++++
 .../master/state/TabletLocationStateTest.java   | 163 +++++++++++++++++
 .../server/problems/ProblemReportTest.java      | 182 +++++++++++++++++++
 .../problems/ProblemReportingIteratorTest.java  |  96 ++++++++++
 .../server/tablets/LogicalTimeTest.java         |  91 ++++++++++
 .../accumulo/server/tablets/MillisTimeTest.java |  87 +++++++++
 .../accumulo/server/tablets/TabletTimeTest.java | 156 ++++++++++++++++
 .../accumulo/server/util/AdminCommandsTest.java |  81 +++++++++
 .../accumulo/server/util/FileInfoTest.java      |  42 +++++
 .../accumulo/server/util/FileUtilTest.java      |  27 ++-
 .../accumulo/server/util/TServerUtilsTest.java  |  77 ++++++++
 15 files changed, 1364 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2089c89/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java
index decc995..b882195 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReport.java
@@ -26,6 +26,7 @@ import java.net.UnknownHostException;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
@@ -39,6 +40,7 @@ import org.apache.accumulo.server.security.SystemCredentials;
 import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.hadoop.io.Text;
+import org.apache.zookeeper.KeeperException;
 
 public class ProblemReport {
   private String tableName;
@@ -48,7 +50,7 @@ public class ProblemReport {
   private String server;
   private long creationTime;
   
-  public ProblemReport(String table, ProblemType problemType, String resource, String server, Throwable e) {
+  public ProblemReport(String table, ProblemType problemType, String resource, String server, Throwable e, long creationTime) {
     ArgumentChecker.notNull(table, problemType, resource);
     this.tableName = table;
     
@@ -68,9 +70,13 @@ public class ProblemReport {
     }
     
     this.server = server;
-    this.creationTime = System.currentTimeMillis();
+    this.creationTime = creationTime;
   }
   
+  public ProblemReport(String table, ProblemType problemType, String resource, String server, Throwable e) {
+    this(table, problemType, resource, server, e, System.currentTimeMillis());
+  }
+
   public ProblemReport(String table, ProblemType problemType, String resource, Throwable e) {
     this(table, problemType, resource, null, e);
   }
@@ -138,15 +144,21 @@ public class ProblemReport {
   }
   
   void removeFromZooKeeper() throws Exception {
-    String zpath = getZPath();
-    ZooReaderWriter.getInstance().recursiveDelete(zpath, NodeMissingPolicy.SKIP);
+    removeFromZooKeeper(ZooReaderWriter.getInstance(), HdfsZooInstance.getInstance());
+  }
+  void removeFromZooKeeper(ZooReaderWriter zoorw, Instance instance) throws IOException, KeeperException, InterruptedException {
+    String zpath = getZPath(instance);
+    zoorw.recursiveDelete(zpath, NodeMissingPolicy.SKIP);
   }
   
   void saveToZooKeeper() throws Exception {
-    ZooReaderWriter.getInstance().putPersistentData(getZPath(), encode(), NodeExistsPolicy.OVERWRITE);
+    saveToZooKeeper(ZooReaderWriter.getInstance(), HdfsZooInstance.getInstance());
+  }
+  void saveToZooKeeper(ZooReaderWriter zoorw, Instance instance) throws IOException, KeeperException, InterruptedException {
+    zoorw.putPersistentData(getZPath(instance), encode(), NodeExistsPolicy.OVERWRITE);
   }
   
-  private String getZPath() throws IOException {
+  private String getZPath(Instance instance) throws IOException {
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
     DataOutputStream dos = new DataOutputStream(baos);
     dos.writeUTF(getTableName());
@@ -155,11 +167,14 @@ public class ProblemReport {
     dos.close();
     baos.close();
     
-    String zpath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZPROBLEMS + "/" + Encoding.encodeAsBase64FileName(new Text(baos.toByteArray()));
+    String zpath = ZooUtil.getRoot(instance) + Constants.ZPROBLEMS + "/" + Encoding.encodeAsBase64FileName(new Text(baos.toByteArray()));
     return zpath;
   }
   
   static ProblemReport decodeZooKeeperEntry(String node) throws Exception {
+    return decodeZooKeeperEntry(node, ZooReaderWriter.getInstance(), HdfsZooInstance.getInstance());
+  }
+  static ProblemReport decodeZooKeeperEntry(String node, ZooReaderWriter zoorw, Instance instance) throws IOException, KeeperException, InterruptedException {
     byte bytes[] = Encoding.decodeBase64FileName(node);
     
     ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
@@ -169,8 +184,8 @@ public class ProblemReport {
     String problemType = dis.readUTF();
     String resource = dis.readUTF();
     
-    String zpath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZPROBLEMS + "/" + node;
-    byte[] enc = ZooReaderWriter.getInstance().getData(zpath, null);
+    String zpath = ZooUtil.getRoot(instance) + Constants.ZPROBLEMS + "/" + node;
+    byte[] enc = zoorw.getData(zpath, null);
     
     return new ProblemReport(tableName, ProblemType.valueOf(problemType), resource, enc);
     

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2089c89/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java b/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java
new file mode 100644
index 0000000..9366163
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/AccumuloTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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;
+
+import java.io.FileNotFoundException;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+
+public class AccumuloTest {
+  private FileSystem fs;
+  private Path path;
+
+  @Before
+  public void setUp() throws Exception {
+    fs = createMock(FileSystem.class);
+    path = createMock(Path.class);
+  }
+
+  private FileStatus[] mockPersistentVersion(String s) {
+    FileStatus[] files = new FileStatus[1];
+    files[0] = createMock(FileStatus.class);
+    Path filePath = createMock(Path.class);
+    expect(filePath.getName()).andReturn(s);
+    replay(filePath);
+    expect(files[0].getPath()).andReturn(filePath);
+    replay(files[0]);
+    return files;
+  }
+
+  @Test
+  public void testGetAccumuloPersistentVersion() throws Exception {
+    FileStatus[] files = mockPersistentVersion("42");
+    expect(fs.listStatus(path)).andReturn(files);
+    replay(fs);
+
+    assertEquals(42, Accumulo.getAccumuloPersistentVersion(fs, path));
+  }
+
+  @Test
+  public void testGetAccumuloPersistentVersion_Null() throws Exception {
+    expect(fs.listStatus(path)).andReturn(null);
+    replay(fs);
+
+    assertEquals(-1, Accumulo.getAccumuloPersistentVersion(fs, path));
+  }
+
+  @Test
+  public void testGetAccumuloPersistentVersion_Empty() throws Exception {
+    expect(fs.listStatus(path)).andReturn(new FileStatus[0]);
+    replay(fs);
+
+    assertEquals(-1, Accumulo.getAccumuloPersistentVersion(fs, path));
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testGetAccumuloPersistentVersion_Fail() throws Exception {
+    expect(fs.listStatus(path)).andThrow(new FileNotFoundException());
+    replay(fs);
+
+    assertEquals(-1, Accumulo.getAccumuloPersistentVersion(fs, path));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2089c89/server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java b/server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java
new file mode 100644
index 0000000..d4420aa
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java
@@ -0,0 +1,42 @@
+/*
+ * 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;
+
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class ServerOptsTest {
+  private ServerOpts opts;
+
+  @Before
+  public void setUp() throws Exception {
+    opts = new ServerOpts();
+  }
+
+  @Test
+  public void testGetAddress() {
+    opts.parseArgs(ServerOptsTest.class.getName(), new String[] {"-a", "1.2.3.4"});
+    assertEquals("1.2.3.4", opts.getAddress());
+  }
+
+  @Test
+  public void testGetAddress_NOne() {
+    opts.parseArgs(ServerOptsTest.class.getName(), new String[] {});
+    assertEquals("0.0.0.0", opts.getAddress());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2089c89/server/base/src/test/java/org/apache/accumulo/server/conf/TableConfigurationTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/conf/TableConfigurationTest.java b/server/base/src/test/java/org/apache/accumulo/server/conf/TableConfigurationTest.java
new file mode 100644
index 0000000..8731de1
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/conf/TableConfigurationTest.java
@@ -0,0 +1,46 @@
+/*
+ * 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.conf;
+
+import org.apache.accumulo.core.client.Instance;
+
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import org.easymock.Capture;
+import static org.easymock.EasyMock.*;
+
+public class TableConfigurationTest {
+  private static final String INSTANCE_ID = "instanceId";
+  private static final String TABLE = "table";
+  private Instance instance;
+  private NamespaceConfiguration nsconf;
+  private TableConfiguration tconf;
+
+  @Before
+  public void setUp() throws Exception {
+    instance = createMock(Instance.class);
+    nsconf = createMock(NamespaceConfiguration.class);
+    tconf = new TableConfiguration(INSTANCE_ID, TABLE, nsconf);
+  }
+
+  @Test
+  public void testGetters() {
+    assertEquals(TABLE, tconf.getTableId());
+    assertSame(nsconf, tconf.getParentConfiguration());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2089c89/server/base/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java
new file mode 100644
index 0000000..58484af
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/state/MergeInfoTest.java
@@ -0,0 +1,166 @@
+/*
+ * 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.master.state;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.expectLastCall;
+import static org.easymock.EasyMock.replay;
+
+public class MergeInfoTest {
+  private KeyExtent keyExtent;
+  private MergeInfo mi;
+
+  @Before
+  public void setUp() throws Exception {
+    keyExtent = createMock(KeyExtent.class);
+  }
+
+  @Test
+  public void testConstruction_NoArgs() {
+    mi = new MergeInfo();
+    assertEquals(MergeState.NONE, mi.getState());
+    assertNull(mi.getExtent());
+    assertEquals(MergeInfo.Operation.MERGE, mi.getOperation());
+    assertFalse(mi.isDelete());
+  }
+
+  @Test
+  public void testConstruction_2Args() {
+    mi = new MergeInfo(keyExtent, MergeInfo.Operation.DELETE);
+    assertEquals(MergeState.NONE, mi.getState());
+    assertSame(keyExtent, mi.getExtent());
+    assertEquals(MergeInfo.Operation.DELETE, mi.getOperation());
+    assertTrue(mi.isDelete());
+  }
+
+  @Test
+  public void testSerialization() throws Exception {
+    Text table = new Text("table");
+    Text endRow = new Text("end");
+    Text prevEndRow = new Text("begin");
+    keyExtent = new KeyExtent(table, endRow, prevEndRow);
+    mi = new MergeInfo(keyExtent, MergeInfo.Operation.DELETE);
+    mi.setState(MergeState.STARTED);
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(baos);
+    mi.write(dos);
+    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+    DataInputStream dis = new DataInputStream(bais);
+    mi = new MergeInfo();
+    mi.readFields(dis);
+    assertSame(MergeState.STARTED, mi.getState());
+    assertEquals(keyExtent, mi.getExtent());
+    assertSame(MergeInfo.Operation.DELETE, mi.getOperation());
+  }
+
+  @Test
+  public void testNeedsToBeChopped_DifferentTables() {
+    expect(keyExtent.getTableId()).andReturn(new Text("table1"));
+    replay(keyExtent);
+    KeyExtent keyExtent2 = createMock(KeyExtent.class);
+    expect(keyExtent2.getTableId()).andReturn(new Text("table2"));
+    replay(keyExtent2);
+    mi = new MergeInfo(keyExtent, MergeInfo.Operation.MERGE);
+    assertFalse(mi.needsToBeChopped(keyExtent2));
+  }
+
+  @Test
+  public void testNeedsToBeChopped_NotDelete() {
+    expect(keyExtent.getTableId()).andReturn(new Text("table1"));
+    KeyExtent keyExtent2 = createMock(KeyExtent.class);
+    expect(keyExtent2.getTableId()).andReturn(new Text("table1"));
+    replay(keyExtent2);
+    expect(keyExtent.overlaps(keyExtent2)).andReturn(true);
+    replay(keyExtent);
+    mi = new MergeInfo(keyExtent, MergeInfo.Operation.MERGE);
+    assertTrue(mi.needsToBeChopped(keyExtent2));
+  }
+
+  @Test
+  public void testNeedsToBeChopped_Delete_NotFollowing() {
+    testNeedsToBeChopped_Delete("somerow", false);
+  }
+
+  @Test
+  public void testNeedsToBeChopped_Delete_Following() {
+    testNeedsToBeChopped_Delete("prev", true);
+  }
+
+  @Test
+  public void testNeedsToBeChopped_Delete_NoPrevEndRow() {
+    testNeedsToBeChopped_Delete(null, false);
+  }
+
+  private void testNeedsToBeChopped_Delete(String prevEndRow, boolean expected) {
+    expect(keyExtent.getTableId()).andReturn(new Text("table1"));
+    expect(keyExtent.getEndRow()).andReturn(new Text("prev"));
+    replay(keyExtent);
+    KeyExtent keyExtent2 = createMock(KeyExtent.class);
+    expect(keyExtent2.getTableId()).andReturn(new Text("table1"));
+    expect(keyExtent2.getPrevEndRow()).andReturn(prevEndRow != null ? new Text(prevEndRow) : null);
+    expectLastCall().anyTimes();
+    replay(keyExtent2);
+    mi = new MergeInfo(keyExtent, MergeInfo.Operation.DELETE);
+    assertEquals(expected, mi.needsToBeChopped(keyExtent2));
+  }
+
+  @Test
+  public void testOverlaps_ExtentsOverlap() {
+    KeyExtent keyExtent2 = createMock(KeyExtent.class);
+    expect(keyExtent.overlaps(keyExtent2)).andReturn(true);
+    replay(keyExtent);
+    mi = new MergeInfo(keyExtent, MergeInfo.Operation.MERGE);
+    assertTrue(mi.overlaps(keyExtent2));
+  }
+
+  @Test
+  public void testOverlaps_DoesNotNeedChopping() {
+    KeyExtent keyExtent2 = createMock(KeyExtent.class);
+    expect(keyExtent.overlaps(keyExtent2)).andReturn(false);
+    expect(keyExtent.getTableId()).andReturn(new Text("table1"));
+    replay(keyExtent);
+    expect(keyExtent2.getTableId()).andReturn(new Text("table2"));
+    replay(keyExtent2);
+    mi = new MergeInfo(keyExtent, MergeInfo.Operation.MERGE);
+    assertFalse(mi.overlaps(keyExtent2));
+  }
+
+  @Test
+  public void testOverlaps_NeedsChopping() {
+    KeyExtent keyExtent2 = createMock(KeyExtent.class);
+    expect(keyExtent.overlaps(keyExtent2)).andReturn(false);
+    expect(keyExtent.getTableId()).andReturn(new Text("table1"));
+    expect(keyExtent.getEndRow()).andReturn(new Text("prev"));
+    replay(keyExtent);
+    expect(keyExtent2.getTableId()).andReturn(new Text("table1"));
+    expect(keyExtent2.getPrevEndRow()).andReturn(new Text("prev"));
+    expectLastCall().anyTimes();
+    replay(keyExtent2);
+    mi = new MergeInfo(keyExtent, MergeInfo.Operation.DELETE);
+    assertTrue(mi.overlaps(keyExtent2));
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2089c89/server/base/src/test/java/org/apache/accumulo/server/master/state/TabletLocationStateTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/state/TabletLocationStateTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/state/TabletLocationStateTest.java
new file mode 100644
index 0000000..c1f312d
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/state/TabletLocationStateTest.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.server.master.state;
+
+import java.util.Collection;
+import java.util.Set;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+
+public class TabletLocationStateTest {
+  private static final Collection<String> innerWalogs = new java.util.HashSet<String>();
+  private static final Collection<Collection<String>> walogs = new java.util.HashSet<Collection<String>>();
+
+  @BeforeClass
+  public static void setUpClass() {
+    walogs.add(innerWalogs);
+    innerWalogs.add("somelog");
+  }
+
+  private KeyExtent keyExtent;
+  private TServerInstance future;
+  private TServerInstance current;
+  private TServerInstance last;
+  private TabletLocationState tls;
+
+  @Before
+  public void setUp() throws Exception {
+    keyExtent = createMock(KeyExtent.class);
+    future = createMock(TServerInstance.class);
+    current = createMock(TServerInstance.class);
+    last = createMock(TServerInstance.class);
+  }
+
+  @Test
+  public void testConstruction_NoFuture() throws Exception {
+    tls = new TabletLocationState(keyExtent, null, current, last, walogs, true);
+    assertSame(keyExtent, tls.extent);
+    assertNull(tls.future);
+    assertSame(current, tls.current);
+    assertSame(last, tls.last);
+    assertSame(walogs, tls.walogs);
+    assertTrue(tls.chopped);
+  }
+
+  @Test
+  public void testConstruction_NoCurrent() throws Exception {
+    tls = new TabletLocationState(keyExtent, future, null, last, walogs, true);
+    assertSame(keyExtent, tls.extent);
+    assertSame(future, tls.future);
+    assertNull(tls.current);
+    assertSame(last, tls.last);
+    assertSame(walogs, tls.walogs);
+    assertTrue(tls.chopped);
+  }
+
+  @Test(expected = TabletLocationState.BadLocationStateException.class)
+  public void testConstruction_FutureAndCurrent() throws Exception {
+    expect(keyExtent.getMetadataEntry()).andReturn(new Text("entry"));
+    replay(keyExtent);
+    try {
+      new TabletLocationState(keyExtent, future, current, last, walogs, true);
+    } catch (TabletLocationState.BadLocationStateException e) {
+      assertEquals(new Text("entry"), e.getEncodedEndRow());
+      throw (e);
+    }
+  }
+
+  @Test
+  public void testConstruction_NoFuture_NoWalogs() throws Exception {
+    tls = new TabletLocationState(keyExtent, null, current, last, null, true);
+    assertNotNull(tls.walogs);
+    assertEquals(0, tls.walogs.size());
+  }
+
+  @Test
+  public void testGetServer_Current() throws Exception {
+    tls = new TabletLocationState(keyExtent, null, current, last, walogs, true);
+    assertSame(current, tls.getServer());
+  }
+
+  @Test
+  public void testGetServer_Future() throws Exception {
+    tls = new TabletLocationState(keyExtent, future, null, last, walogs, true);
+    assertSame(future, tls.getServer());
+  }
+
+  @Test
+  public void testGetServer_Last() throws Exception {
+    tls = new TabletLocationState(keyExtent, null, null, last, walogs, true);
+    assertSame(last, tls.getServer());
+  }
+
+  @Test
+  public void testGetServer_None() throws Exception {
+    tls = new TabletLocationState(keyExtent, null, null, null, walogs, true);
+    assertNull(tls.getServer());
+  }
+
+  @Test
+  public void testGetState_Unassigned1() throws Exception {
+    tls = new TabletLocationState(keyExtent, null, null, null, walogs, true);
+    assertEquals(TabletState.UNASSIGNED, tls.getState(null));
+  }
+
+  @Test
+  public void testGetState_Unassigned2() throws Exception {
+    tls = new TabletLocationState(keyExtent, null, null, last, walogs, true);
+    assertEquals(TabletState.UNASSIGNED, tls.getState(null));
+  }
+
+  @Test
+  public void testGetState_Assigned() throws Exception {
+    Set<TServerInstance> liveServers = new java.util.HashSet<TServerInstance>();
+    liveServers.add(future);
+    tls = new TabletLocationState(keyExtent, future, null, last, walogs, true);
+    assertEquals(TabletState.ASSIGNED, tls.getState(liveServers));
+  }
+
+  @Test
+  public void testGetState_Hosted() throws Exception {
+    Set<TServerInstance> liveServers = new java.util.HashSet<TServerInstance>();
+    liveServers.add(current);
+    tls = new TabletLocationState(keyExtent, null, current, last, walogs, true);
+    assertEquals(TabletState.HOSTED, tls.getState(liveServers));
+  }
+
+  @Test
+  public void testGetState_Dead1() throws Exception {
+    Set<TServerInstance> liveServers = new java.util.HashSet<TServerInstance>();
+    liveServers.add(current);
+    tls = new TabletLocationState(keyExtent, future, null, last, walogs, true);
+    assertEquals(TabletState.ASSIGNED_TO_DEAD_SERVER, tls.getState(liveServers));
+  }
+
+  @Test
+  public void testGetState_Dead2() throws Exception {
+    Set<TServerInstance> liveServers = new java.util.HashSet<TServerInstance>();
+    liveServers.add(future);
+    tls = new TabletLocationState(keyExtent, null, current, last, walogs, true);
+    assertEquals(TabletState.ASSIGNED_TO_DEAD_SERVER, tls.getState(liveServers));
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2089c89/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java
new file mode 100644
index 0000000..dbad326
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportTest.java
@@ -0,0 +1,182 @@
+/*
+ * 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.problems;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.util.Encoding;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
+import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
+import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
+import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import static org.easymock.EasyMock.aryEq;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.eq;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+
+public class ProblemReportTest {
+  private static final String TABLE = "table";
+  private static final String RESOURCE = "resource";
+  private static final String SERVER = "server";
+
+  private Instance instance;
+  private ZooReaderWriter zoorw;
+  private ProblemReport r;
+
+  @Before
+  public void setUp() throws Exception {
+    instance = createMock(Instance.class);
+    expect(instance.getInstanceID()).andReturn("instance");
+    replay(instance);
+
+    zoorw = createMock(ZooReaderWriter.class);
+  }
+
+  @Test
+  public void testGetters() {
+    long now = System.currentTimeMillis();
+    r = new ProblemReport(TABLE, ProblemType.FILE_READ, RESOURCE, SERVER, null, now);
+    assertEquals(TABLE, r.getTableName());
+    assertSame(ProblemType.FILE_READ, r.getProblemType());
+    assertEquals(RESOURCE, r.getResource());
+    assertEquals(SERVER, r.getServer());
+    assertEquals(now, r.getTime());
+    assertNull(r.getException());
+  }
+
+  @Test
+  public void testWithException() {
+    Exception e = new IllegalArgumentException("Oh noes");
+    r = new ProblemReport(TABLE, ProblemType.FILE_READ, RESOURCE, SERVER, e);
+    assertEquals("Oh noes", r.getException());
+  }
+
+  @Test
+  public void testEquals() {
+    r = new ProblemReport(TABLE, ProblemType.FILE_READ, RESOURCE, SERVER, null);
+    assertTrue(r.equals(r));
+    ProblemReport r2 = new ProblemReport(TABLE, ProblemType.FILE_READ, RESOURCE, SERVER, null);
+    assertTrue(r.equals(r2));
+    assertTrue(r2.equals(r));
+    ProblemReport rx1 = new ProblemReport(TABLE + "x", ProblemType.FILE_READ, RESOURCE, SERVER, null);
+    assertFalse(r.equals(rx1));
+    ProblemReport rx2 = new ProblemReport(TABLE, ProblemType.FILE_WRITE, RESOURCE, SERVER, null);
+    assertFalse(r.equals(rx2));
+    ProblemReport rx3 = new ProblemReport(TABLE, ProblemType.FILE_READ, RESOURCE + "x", SERVER, null);
+    assertFalse(r.equals(rx3));
+    ProblemReport re1 = new ProblemReport(TABLE, ProblemType.FILE_READ, RESOURCE, SERVER + "x", null);
+    assertTrue(r.equals(re1));
+    ProblemReport re2 = new ProblemReport(TABLE, ProblemType.FILE_READ, RESOURCE, SERVER, new IllegalArgumentException("yikes"));
+    assertTrue(r.equals(re2));
+  }
+
+  @Test
+  public void testEqualsNull() {
+    r = new ProblemReport(TABLE, ProblemType.FILE_READ, RESOURCE, SERVER, null);
+    assertFalse(r.equals(null));
+  }
+
+  @Test
+  public void testHashCode() {
+    r = new ProblemReport(TABLE, ProblemType.FILE_READ, RESOURCE, SERVER, null);
+    ProblemReport r2 = new ProblemReport(TABLE, ProblemType.FILE_READ, RESOURCE, SERVER, null);
+    assertEquals(r.hashCode(), r2.hashCode());
+    ProblemReport re1 = new ProblemReport(TABLE, ProblemType.FILE_READ, RESOURCE, SERVER + "x", null);
+    assertEquals(r.hashCode(), re1.hashCode());
+    ProblemReport re2 = new ProblemReport(TABLE, ProblemType.FILE_READ, RESOURCE, SERVER, new IllegalArgumentException("yikes"));
+    assertEquals(r.hashCode(), re2.hashCode());
+  }
+
+  private byte[] makeZPathFileName(String table, ProblemType problemType, String resource) throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(baos);
+    dos.writeUTF(table);
+    dos.writeUTF(problemType.name());
+    dos.writeUTF(resource);
+    dos.close();
+    return baos.toByteArray();
+  }
+
+  private byte[] encodeReportData(long creationTime, String server, String exception) throws Exception {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(baos);
+    dos.writeLong(creationTime);
+    dos.writeBoolean(server != null);
+    if (server != null) {
+      dos.writeUTF(server);
+    }
+    dos.writeBoolean(exception != null);
+    if (exception != null) {
+      dos.writeUTF(exception);
+    }
+    dos.close();
+    return baos.toByteArray();
+  }
+
+  @Test
+  public void testRemoveFromZooKeeper() throws Exception {
+    r = new ProblemReport(TABLE, ProblemType.FILE_READ, RESOURCE, SERVER, null);
+    byte[] zpathFileName = makeZPathFileName(TABLE, ProblemType.FILE_READ, RESOURCE);
+    String path = ZooUtil.getRoot("instance") + Constants.ZPROBLEMS + "/" + Encoding.encodeAsBase64FileName(new Text(zpathFileName));
+    zoorw.recursiveDelete(path, NodeMissingPolicy.SKIP);
+    replay(zoorw);
+
+    r.removeFromZooKeeper(zoorw, instance);
+    verify(zoorw);
+  }
+
+  @Test
+  public void testSaveToZooKeeper() throws Exception {
+    long now = System.currentTimeMillis();
+    r = new ProblemReport(TABLE, ProblemType.FILE_READ, RESOURCE, SERVER, null, now);
+    byte[] zpathFileName = makeZPathFileName(TABLE, ProblemType.FILE_READ, RESOURCE);
+    String path = ZooUtil.getRoot("instance") + Constants.ZPROBLEMS + "/" + Encoding.encodeAsBase64FileName(new Text(zpathFileName));
+    byte[] encoded = encodeReportData(now, SERVER, null);
+    expect(zoorw.putPersistentData(eq(path), aryEq(encoded), eq(NodeExistsPolicy.OVERWRITE))).andReturn(true);
+    replay(zoorw);
+
+    r.saveToZooKeeper(zoorw, instance);
+    verify(zoorw);
+  }
+
+  @Test
+  public void testDecodeZooKeeperEntry() throws Exception {
+    byte[] zpathFileName = makeZPathFileName(TABLE, ProblemType.FILE_READ, RESOURCE);
+    String node = Encoding.encodeAsBase64FileName(new Text(zpathFileName));
+    long now = System.currentTimeMillis();
+    byte[] encoded = encodeReportData(now, SERVER, "excmsg");
+
+    expect(zoorw.getData(ZooUtil.getRoot("instance") + Constants.ZPROBLEMS + "/" + node, null)).andReturn(encoded);
+    replay(zoorw);
+
+    r = ProblemReport.decodeZooKeeperEntry(node, zoorw, instance);
+    assertEquals(TABLE, r.getTableName());
+    assertSame(ProblemType.FILE_READ, r.getProblemType());
+    assertEquals(RESOURCE, r.getResource());
+    assertEquals(SERVER, r.getServer());
+    assertEquals(now, r.getTime());
+    assertEquals("excmsg", r.getException());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2089c89/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
new file mode 100644
index 0000000..eab9614
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/problems/ProblemReportingIteratorTest.java
@@ -0,0 +1,96 @@
+/*
+ * 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.problems;
+
+import java.util.Collection;
+import java.util.concurrent.atomic.AtomicBoolean;
+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.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+
+public class ProblemReportingIteratorTest {
+  private static final String TABLE = "table";
+  private static final String RESOURCE = "resource";
+  private static final String SERVER = "server";
+
+  private InterruptibleIterator ii;
+  private ProblemReportingIterator pri;
+
+  @Before
+  public void setUp() throws Exception {
+    ii = createMock(InterruptibleIterator.class);
+    pri = new ProblemReportingIterator(TABLE, RESOURCE, false, ii);
+  }
+
+  @Test
+  public void testBasicGetters() {
+    Key key = createMock(Key.class);
+    expect(ii.getTopKey()).andReturn(key);
+    Value value = createMock(Value.class);
+    expect(ii.getTopValue()).andReturn(value);
+    expect(ii.hasTop()).andReturn(true);
+    replay(ii);
+    assertSame(key, pri.getTopKey());
+    assertSame(value, pri.getTopValue());
+    assertTrue(pri.hasTop());
+    assertFalse(pri.sawError());
+    assertEquals(RESOURCE, pri.getResource());
+  }
+
+  @Test(expected = UnsupportedOperationException.class)
+  public void testInit() throws Exception {
+    pri.init(null, null, null);
+  }
+
+  @Test
+  public void testNext() throws Exception {
+    ii.next();
+    replay(ii);
+    pri.next();
+    verify(ii);
+    assertFalse(pri.sawError());
+  }
+
+  @Test
+  public void testSeek() throws Exception {
+    Range r = createMock(Range.class);
+    Collection<ByteSequence> f = new java.util.HashSet<ByteSequence>();
+    ii.seek(r, f, true);
+    replay(ii);
+    pri.seek(r, f, true);
+    verify(ii);
+    assertFalse(pri.sawError());
+  }
+
+  @Test
+  public void testSetInterruptFlag() {
+    AtomicBoolean flag = new AtomicBoolean(true);
+    ii.setInterruptFlag(flag);
+    replay(ii);
+    pri.setInterruptFlag(flag);
+    verify(ii);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2089c89/server/base/src/test/java/org/apache/accumulo/server/tablets/LogicalTimeTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/tablets/LogicalTimeTest.java b/server/base/src/test/java/org/apache/accumulo/server/tablets/LogicalTimeTest.java
new file mode 100644
index 0000000..598eb79
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/tablets/LogicalTimeTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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.tablets;
+
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.server.data.ServerMutation;
+import org.apache.accumulo.server.tablets.TabletTime.LogicalTime;
+
+import java.util.List;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import static org.easymock.EasyMock.anyLong;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+
+public class LogicalTimeTest {
+  private static final long TIME = 1234L;
+  private LogicalTime ltime;
+
+  @Before
+  public void setUp() throws Exception {
+    ltime = (LogicalTime) TabletTime.getInstance("L1234");
+  }
+
+  @Test
+  public void testGetMetadataValue() {
+    assertEquals("L1234", ltime.getMetadataValue());
+  }
+
+  @Test
+  public void testUseMaxTimeFromWALog_Update() {
+    ltime.useMaxTimeFromWALog(5678L);
+    assertEquals("L5678", ltime.getMetadataValue());
+  }
+
+  @Test
+  public void testUseMaxTimeFromWALog_NoUpdate() {
+    ltime.useMaxTimeFromWALog(0L);
+    assertEquals("L1234", ltime.getMetadataValue());
+  }
+
+  @Test
+  public void testSetUpdateTimes() {
+    List<Mutation> ms = new java.util.ArrayList<Mutation>();
+    ServerMutation m = createMock(ServerMutation.class);
+    ServerMutation m2 = createMock(ServerMutation.class);
+    m.setSystemTimestamp(1235L);
+    replay(m);
+    m2.setSystemTimestamp(1236L);
+    replay(m2);
+    ms.add(m);
+    ms.add(m2);
+    long currTime = ltime.setUpdateTimes(ms);
+    assertEquals(TIME + 2L, currTime);
+    verify(m);
+    verify(m2);
+  }
+
+  @Test
+  public void testSetUpdateTimes_NoMutations() {
+    List<Mutation> ms = new java.util.ArrayList<Mutation>();
+    assertEquals(TIME, ltime.setUpdateTimes(ms));
+  }
+
+  @Test
+  public void testGetTime() {
+    assertEquals(TIME, ltime.getTime());
+  }
+
+  @Test
+  public void testGetAndUpdateTime() {
+    long currTime = ltime.getAndUpdateTime();
+    assertEquals(TIME + 1L, currTime);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2089c89/server/base/src/test/java/org/apache/accumulo/server/tablets/MillisTimeTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/tablets/MillisTimeTest.java b/server/base/src/test/java/org/apache/accumulo/server/tablets/MillisTimeTest.java
new file mode 100644
index 0000000..15c8465
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/tablets/MillisTimeTest.java
@@ -0,0 +1,87 @@
+/*
+ * 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.tablets;
+
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.server.data.ServerMutation;
+import org.apache.accumulo.server.tablets.TabletTime.MillisTime;
+
+import java.util.List;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import static org.easymock.EasyMock.anyLong;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+
+public class MillisTimeTest {
+  private static final long TIME = 1234L;
+  private MillisTime mtime;
+
+  @Before
+  public void setUp() throws Exception {
+    mtime = new MillisTime(TIME);
+  }
+
+  @Test
+  public void testGetMetadataValue() {
+    assertEquals("M1234", mtime.getMetadataValue());
+  }
+
+  @Test
+  public void testUseMaxTimeFromWALog_Yes() {
+    mtime.useMaxTimeFromWALog(5678L);
+    assertEquals("M5678", mtime.getMetadataValue());
+  }
+
+  @Test
+  public void testUseMaxTimeFromWALog_No() {
+    mtime.useMaxTimeFromWALog(0L);
+    assertEquals("M1234", mtime.getMetadataValue());
+  }
+
+  @Test
+  public void testSetUpdateTimes() {
+    List<Mutation> ms = new java.util.ArrayList<Mutation>();
+    ServerMutation m = createMock(ServerMutation.class);
+    m.setSystemTimestamp(anyLong());
+    replay(m);
+    ms.add(m);
+    long currTime = mtime.setUpdateTimes(ms);
+    assertTrue(currTime > TIME);
+    verify(m);
+  }
+
+  @Test
+  public void testSetUpdateTimes_NoMutations() {
+    List<Mutation> ms = new java.util.ArrayList<Mutation>();
+    assertEquals(TIME, mtime.setUpdateTimes(ms));
+  }
+
+  @Test
+  public void testGetTime() {
+    assertEquals(TIME, mtime.getTime());
+  }
+
+  @Test
+  public void testGetAndUpdateTime() {
+    long currTime = mtime.getAndUpdateTime();
+    assertTrue(currTime > TIME);
+    assertEquals(currTime, mtime.getTime());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2089c89/server/base/src/test/java/org/apache/accumulo/server/tablets/TabletTimeTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/tablets/TabletTimeTest.java b/server/base/src/test/java/org/apache/accumulo/server/tablets/TabletTimeTest.java
new file mode 100644
index 0000000..38cca85
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/tablets/TabletTimeTest.java
@@ -0,0 +1,156 @@
+/*
+ * 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.tablets;
+
+import org.apache.accumulo.core.client.admin.TimeType;
+import org.apache.accumulo.server.data.ServerMutation;
+import org.apache.accumulo.server.tablets.TabletTime.LogicalTime;
+import org.apache.accumulo.server.tablets.TabletTime.MillisTime;
+
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+
+public class TabletTimeTest {
+  private static final long TIME = 1234L;
+  private MillisTime mtime;
+
+  @Before
+  public void setUp() throws Exception {
+    mtime = new MillisTime(TIME);
+  }
+
+  @Test
+  public void testGetTimeID() {
+    assertEquals('L', TabletTime.getTimeID(TimeType.LOGICAL));
+    assertEquals('M', TabletTime.getTimeID(TimeType.MILLIS));
+  }
+
+  @Test
+  public void testSetSystemTimes() {
+    ServerMutation m = createMock(ServerMutation.class);
+    long lastCommitTime = 1234L;
+    m.setSystemTimestamp(lastCommitTime);
+    replay(m);
+    mtime.setSystemTimes(m, lastCommitTime);
+    verify(m);
+  }
+
+  @Test
+  public void testGetInstance_Logical() {
+    TabletTime t = TabletTime.getInstance("L1234");
+    assertEquals(LogicalTime.class, t.getClass());
+    assertEquals("L1234", t.getMetadataValue());
+  }
+
+  @Test
+  public void testGetInstance_Millis() {
+    TabletTime t = TabletTime.getInstance("M1234");
+    assertEquals(MillisTime.class, t.getClass());
+    assertEquals("M1234", t.getMetadataValue());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testGetInstance_InvalidType() {
+    TabletTime.getInstance("X1234");
+  }
+
+  @Test(expected = NumberFormatException.class)
+  public void testGetInstance_Logical_ParseFailure() {
+    TabletTime.getInstance("LABCD");
+  }
+
+  @Test(expected = NumberFormatException.class)
+  public void testGetInstance_Millis_ParseFailure() {
+    TabletTime.getInstance("MABCD");
+  }
+
+  @Test
+  public void testMaxMetadataTime_Logical() {
+    assertEquals("L5678", TabletTime.maxMetadataTime("L1234", "L5678"));
+    assertEquals("L5678", TabletTime.maxMetadataTime("L5678", "L1234"));
+    assertEquals("L5678", TabletTime.maxMetadataTime("L5678", "L5678"));
+  }
+
+  @Test
+  public void testMaxMetadataTime_Millis() {
+    assertEquals("M5678", TabletTime.maxMetadataTime("M1234", "M5678"));
+    assertEquals("M5678", TabletTime.maxMetadataTime("M5678", "M1234"));
+    assertEquals("M5678", TabletTime.maxMetadataTime("M5678", "M5678"));
+  }
+
+  @Test
+  public void testMaxMetadataTime_Null1() {
+    assertEquals("L5678", TabletTime.maxMetadataTime(null, "L5678"));
+    assertEquals("M5678", TabletTime.maxMetadataTime(null, "M5678"));
+  }
+
+  @Test
+  public void testMaxMetadataTime_Null2() {
+    assertEquals("L5678", TabletTime.maxMetadataTime("L5678", null));
+    assertEquals("M5678", TabletTime.maxMetadataTime("M5678", null));
+  }
+
+  @Test
+  @org.junit.Ignore
+  public void testMaxMetadataTime_Null3() {
+    assertNull(TabletTime.maxMetadataTime(null, null));
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testMaxMetadataTime_Null1_Invalid() {
+    TabletTime.maxMetadataTime(null, "X5678");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testMaxMetadataTime_Null2_Invalid() {
+    TabletTime.maxMetadataTime("X5678", null);
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testMaxMetadataTime_Invalid1() {
+    TabletTime.maxMetadataTime("X1234", "L5678");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testMaxMetadataTime_Invalid2() {
+    TabletTime.maxMetadataTime("L1234", "X5678");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testMaxMetadataTime_DifferentTypes1() {
+    TabletTime.maxMetadataTime("L1234", "M5678");
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testMaxMetadataTime_DifferentTypes2() {
+    TabletTime.maxMetadataTime("X1234", "Y5678");
+  }
+
+  @Test(expected = NumberFormatException.class)
+  public void testMaxMetadataTime_ParseFailure1() {
+    TabletTime.maxMetadataTime("L1234", "LABCD");
+  }
+
+  @Test(expected = NumberFormatException.class)
+  public void testMaxMetadataTime_ParseFailure2() {
+    TabletTime.maxMetadataTime("LABCD", "L5678");
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2089c89/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java
new file mode 100644
index 0000000..3802983
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/AdminCommandsTest.java
@@ -0,0 +1,81 @@
+/*
+ * 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 org.junit.Test;
+import static org.junit.Assert.*;
+
+public class AdminCommandsTest {
+  @Test
+  public void testStopCommand() {
+    Admin.StopCommand cmd = new Admin.StopCommand();
+    assertEquals(0, cmd.args.size());
+  }
+
+  @Test
+  public void testPingCommand() {
+    Admin.PingCommand cmd = new Admin.PingCommand();
+    assertEquals(0, cmd.args.size());
+  }
+
+  @Test
+  public void testCheckTabletsCommand() {
+    Admin.CheckTabletsCommand cmd = new Admin.CheckTabletsCommand();
+    assertFalse(cmd.fixFiles);
+    assertNull(cmd.table);
+  }
+
+  @Test
+  public void testStopMasterCommand() {
+    new Admin.StopMasterCommand();
+  }
+
+  @Test
+  public void testStopAllCommand() {
+    new Admin.StopAllCommand();
+  }
+
+  @Test
+  public void testListInstancesCommand() {
+    Admin.ListInstancesCommand cmd = new Admin.ListInstancesCommand();
+    assertFalse(cmd.printErrors);
+    assertFalse(cmd.printAll);
+  }
+
+  @Test
+  public void testVolumesCommand() {
+    Admin.VolumesCommand cmd = new Admin.VolumesCommand();
+    assertFalse(cmd.printErrors);
+  }
+
+  @Test
+  public void testDumpConfigCommand() {
+    Admin.DumpConfigCommand cmd = new Admin.DumpConfigCommand();
+    assertEquals(0, cmd.tables.size());
+    assertFalse(cmd.allConfiguration);
+    assertFalse(cmd.systemConfiguration);
+    assertFalse(cmd.userPermissions);
+    assertNull(cmd.directory);
+  }
+
+  // not a command, but easy enough to include here
+  @Test
+  public void testAdminOpts() {
+    Admin.AdminOpts opts = new Admin.AdminOpts();
+    assertFalse(opts.force);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2089c89/server/base/src/test/java/org/apache/accumulo/server/util/FileInfoTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/FileInfoTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/FileInfoTest.java
new file mode 100644
index 0000000..cd568cf
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/FileInfoTest.java
@@ -0,0 +1,42 @@
+/*
+ * 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 org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.server.util.FileUtil.FileInfo;
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class FileInfoTest {
+  private Key key1;
+  private Key key2;
+  private FileInfo info;
+
+  @Before
+  public void setUp() {
+    key1 = new Key("row1");
+    key2 = new Key("row2");
+    info = new FileInfo(key1, key2);
+  }
+
+  @Test
+  public void testGetters() {
+    assertEquals("row1", info.getFirstRow().toString());
+    assertEquals("row2", info.getLastRow().toString());
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2089c89/server/base/src/test/java/org/apache/accumulo/server/util/FileUtilTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/FileUtilTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/FileUtilTest.java
index 86678e1..8924738 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/FileUtilTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/FileUtilTest.java
@@ -19,13 +19,16 @@ package org.apache.accumulo.server.util;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.Map;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.file.FileSKVIterator;
+import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.commons.io.FileUtils;
@@ -35,11 +38,34 @@ import org.junit.Test;
 
 import com.google.common.io.Files;
 
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.replay;
+
 /**
  * 
  */
 public class FileUtilTest {
 
+  @Test
+  public void testToPathStrings() {
+    Collection<FileRef> c = new java.util.ArrayList<FileRef>();
+    FileRef r1 = createMock(FileRef.class);
+    expect(r1.path()).andReturn(new Path("/foo"));
+    replay(r1);
+    c.add(r1);
+    FileRef r2 = createMock(FileRef.class);
+    expect(r2.path()).andReturn(new Path("/bar"));
+    replay(r2);
+    c.add(r2);
+
+    Collection<String> cs = FileUtil.toPathStrings(c);
+    Assert.assertEquals(2, cs.size());
+    Iterator<String> iter = cs.iterator();
+    Assert.assertEquals("/foo", iter.next());
+    Assert.assertEquals("/bar", iter.next());
+  }
+
   @SuppressWarnings("deprecation")
   @Test
   public void testCleanupIndexOpWithDfsDir() throws IOException {
@@ -230,6 +256,5 @@ public class FileUtilTest {
     public void getProperties(Map<String,String> props, PropertyFilter filter) {
       throw new UnsupportedOperationException();
     }
-
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d2089c89/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java
new file mode 100644
index 0000000..a822b92
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/TServerUtilsTest.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.server.util;
+
+import java.util.concurrent.ExecutorService;
+import org.apache.thrift.server.TServer;
+import org.apache.thrift.transport.TServerSocket;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import static org.easymock.EasyMock.*;
+
+public class TServerUtilsTest {
+  private static class TServerWithoutES extends TServer {
+    boolean stopCalled;
+
+    TServerWithoutES(TServerSocket socket) {
+      super(new TServer.Args(socket));
+      stopCalled = false;
+    }
+
+    @Override
+    public void serve() {}
+
+    @Override
+    public void stop() {
+      stopCalled = true;
+    }
+  }
+
+  private static class TServerWithES extends TServerWithoutES {
+    final ExecutorService executorService_;
+
+    TServerWithES(TServerSocket socket) {
+      super(socket);
+      executorService_ = createMock(ExecutorService.class);
+      expect(executorService_.shutdownNow()).andReturn(null);
+      replay(executorService_);
+    }
+  }
+
+  @Test
+  public void testStopTServer_ES() {
+    TServerSocket socket = createNiceMock(TServerSocket.class);
+    TServerWithES s = new TServerWithES(socket);
+    TServerUtils.stopTServer(s);
+    assertTrue(s.stopCalled);
+    verify(s.executorService_);
+  }
+
+  @Test
+  public void testStopTServer_NoES() {
+    TServerSocket socket = createNiceMock(TServerSocket.class);
+    TServerWithoutES s = new TServerWithoutES(socket);
+    TServerUtils.stopTServer(s);
+    assertTrue(s.stopCalled);
+  }
+
+  @Test
+  public void testStopTServer_Null() {
+    TServerUtils.stopTServer(null);
+    // not dying is enough
+  }
+}