You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ed...@apache.org on 2015/07/19 02:25:24 UTC

accumulo git commit: ACCUMULO-3840 replaced OpTimer with code based on hadoop 2.7 org.apache.hadoop.util.StopWatch to remove direct log4j dependicies inherent in OpTimer. Added scale method to provide conversion without truncation that is not in StopWatc

Repository: accumulo
Updated Branches:
  refs/heads/master 4f3d58d0d -> 6977629ed


ACCUMULO-3840 replaced OpTimer with code based on hadoop 2.7 org.apache.hadoop.util.StopWatch to remove direct log4j dependicies inherent in OpTimer. Added scale method to provide conversion without truncation that is not in StopWatch.


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

Branch: refs/heads/master
Commit: 6977629ed404751ba62499595d7f6daacc15ad9a
Parents: 4f3d58d
Author: Ed Coleman <de...@etcoleman.com>
Authored: Fri May 22 10:02:35 2015 -0400
Committer: Ed Coleman <de...@etcoleman.com>
Committed: Sat Jul 18 19:42:07 2015 -0400

----------------------------------------------------------------------
 .../accumulo/core/client/ZooKeeperInstance.java |  37 +++-
 .../client/impl/NamespaceOperationsImpl.java    |  41 +++-
 .../core/client/impl/RootTabletLocator.java     |  21 +-
 .../core/client/impl/TableOperationsImpl.java   |  75 ++++---
 .../core/client/impl/TabletLocatorImpl.java     |  68 ++++---
 .../impl/TabletServerBatchReaderIterator.java   |  71 ++++---
 .../core/client/impl/ThriftScanner.java         |  84 +++++---
 .../core/metadata/MetadataLocationObtainer.java |  39 ++--
 .../org/apache/accumulo/core/util/OpTimer.java  | 135 +++++++++---
 .../apache/accumulo/core/util/OpTimerTest.java  | 203 +++++++++++++++++++
 .../accumulo/server/client/HdfsZooInstance.java |  36 +++-
 .../accumulo/test/NativeMapStressTest.java      |  61 ++++--
 12 files changed, 688 insertions(+), 183 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/6977629e/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
index c5cb482..28f3604 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java
@@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty;
@@ -43,8 +44,8 @@ import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.commons.configuration.Configuration;
 import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * <p>
@@ -62,7 +63,7 @@ import org.apache.log4j.Logger;
 
 public class ZooKeeperInstance implements Instance {
 
-  private static final Logger log = Logger.getLogger(ZooKeeperInstance.class);
+  private static final Logger log = LoggerFactory.getLogger(ZooKeeperInstance.class);
 
   private String instanceId = null;
   private String instanceName = null;
@@ -187,9 +188,20 @@ public class ZooKeeperInstance implements Instance {
   public List<String> getMasterLocations() {
     String masterLocPath = ZooUtil.getRoot(this) + Constants.ZMASTER_LOCK;
 
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up master location in zoocache.");
+    OpTimer timer = null;
+
+    if (log.isTraceEnabled()) {
+      log.trace("tid={} Looking up master location in zookeeper.", Thread.currentThread().getId());
+      timer = new OpTimer().start();
+    }
+
     byte[] loc = ZooUtil.getLockData(zooCache, masterLocPath);
-    opTimer.stop("Found master at " + (loc == null ? null : new String(loc, UTF_8)) + " in %DURATION%");
+
+    if (timer != null) {
+      timer.stop();
+      log.trace("tid={} Found master at {} in {}", Thread.currentThread().getId(), (loc == null ? "null" : new String(loc, UTF_8)),
+          String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+    }
 
     if (loc == null) {
       return Collections.emptyList();
@@ -202,9 +214,20 @@ public class ZooKeeperInstance implements Instance {
   public String getRootTabletLocation() {
     String zRootLocPath = ZooUtil.getRoot(this) + RootTable.ZROOT_TABLET_LOCATION;
 
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up root tablet location in zookeeper.");
+    OpTimer timer = null;
+
+    if (log.isTraceEnabled()) {
+      log.trace("tid={} Looking up root tablet location in zookeeper.", Thread.currentThread().getId());
+      timer = new OpTimer().start();
+    }
+
     byte[] loc = zooCache.get(zRootLocPath);
-    opTimer.stop("Found root tablet at " + (loc == null ? null : new String(loc, UTF_8)) + " in %DURATION%");
+
+    if (timer != null) {
+      timer.stop();
+      log.trace("tid={} Found root tablet at {} in {}", Thread.currentThread().getId(), (loc == null ? "null" : new String(loc, UTF_8)),
+          String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+    }
 
     if (loc == null) {
       return null;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6977629e/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
index b087c73..ae5fc81 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/NamespaceOperationsImpl.java
@@ -29,6 +29,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -50,14 +51,14 @@ import org.apache.accumulo.core.master.thrift.FateOperation;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.trace.Tracer;
 import org.apache.accumulo.core.util.OpTimer;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.Logger;
 
 public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   private final ClientContext context;
   private TableOperationsImpl tableOps;
 
-  private static final Logger log = Logger.getLogger(TableOperations.class);
+  private static final Logger log = LoggerFactory.getLogger(TableOperations.class);
 
   public NamespaceOperationsImpl(ClientContext context, TableOperationsImpl tableOps) {
     checkArgument(context != null, "context is null");
@@ -67,9 +68,22 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
 
   @Override
   public SortedSet<String> list() {
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Fetching list of namespaces...");
+
+    OpTimer timer = null;
+
+    if (log.isTraceEnabled()) {
+      log.trace("tid={} Fetching list of namespaces...", Thread.currentThread().getId());
+      timer = new OpTimer().start();
+    }
+
     TreeSet<String> namespaces = new TreeSet<String>(Namespaces.getNameToIdMap(context.getInstance()).keySet());
-    opTimer.stop("Fetched " + namespaces.size() + " namespaces in %DURATION%");
+
+    if (timer != null) {
+      timer.stop();
+      log.trace("tid={} Fetched {} namespaces in {}", Thread.currentThread().getId(), namespaces.size(),
+          String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+    }
+
     return namespaces;
   }
 
@@ -77,9 +91,20 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
   public boolean exists(String namespace) {
     checkArgument(namespace != null, "namespace is null");
 
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Checking if namespace " + namespace + " exists...");
+    OpTimer timer = null;
+
+    if (log.isTraceEnabled()) {
+      log.trace("tid={} Checking if namespace {} exists", Thread.currentThread().getId(), namespace);
+      timer = new OpTimer().start();
+    }
+
     boolean exists = Namespaces.getNameToIdMap(context.getInstance()).containsKey(namespace);
-    opTimer.stop("Checked existance of " + exists + " in %DURATION%");
+
+    if (timer != null) {
+      timer.stop();
+      log.trace("tid={} Checked existance of {} in {}", Thread.currentThread().getId(), exists, String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+    }
+
     return exists;
   }
 
@@ -103,7 +128,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper {
 
     if (namespaceId.equals(Namespaces.ACCUMULO_NAMESPACE_ID) || namespaceId.equals(Namespaces.DEFAULT_NAMESPACE_ID)) {
       Credentials credentials = context.getCredentials();
-      log.debug(credentials.getPrincipal() + " attempted to delete the " + namespaceId + " namespace");
+      log.debug("{} attempted to delete the {} namespace", credentials.getPrincipal(), namespaceId);
       throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.UNSUPPORTED_OPERATION);
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6977629e/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
index 0992517..4df2e08 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/RootTabletLocator.java
@@ -37,8 +37,8 @@ import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
 import org.apache.accumulo.fate.zookeeper.ZooCacheFactory;
 import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
@@ -106,9 +106,22 @@ public class RootTabletLocator extends TabletLocator {
     String zRootLocPath = ZooUtil.getRoot(instance) + RootTable.ZROOT_TABLET_LOCATION;
     ZooCache zooCache = zcf.getZooCache(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
 
-    OpTimer opTimer = new OpTimer(Logger.getLogger(this.getClass()), Level.TRACE).start("Looking up root tablet location in zookeeper.");
+    Logger log = LoggerFactory.getLogger(this.getClass());
+
+    OpTimer timer = null;
+
+    if (log.isTraceEnabled()) {
+      log.trace("tid={} Looking up root tablet location in zookeeper.", Thread.currentThread().getId());
+      timer = new OpTimer().start();
+    }
+
     byte[] loc = zooCache.get(zRootLocPath);
-    opTimer.stop("Found root tablet at " + (loc == null ? null : new String(loc)) + " in %DURATION%");
+
+    if (timer != null) {
+      timer.stop();
+      log.trace("tid={} Found root tablet at {} in {}", Thread.currentThread().getId(), (loc == null ? "null" : new String(loc)),
+          String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+    }
 
     if (loc == null) {
       return null;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6977629e/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
index e6aecf2..2a486fb 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java
@@ -109,11 +109,11 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
 import org.apache.thrift.TApplicationException;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Joiner;
 import com.google.common.net.HostAndPort;
@@ -122,7 +122,7 @@ import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterrup
 public class TableOperationsImpl extends TableOperationsHelper {
 
   public static final String CLONE_EXCLUDE_PREFIX = "!";
-  private static final Logger log = Logger.getLogger(TableOperations.class);
+  private static final Logger log = LoggerFactory.getLogger(TableOperations.class);
   private final ClientContext context;
 
   public TableOperationsImpl(ClientContext context) {
@@ -132,9 +132,22 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
   @Override
   public SortedSet<String> list() {
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Fetching list of tables...");
+
+    OpTimer timer = null;
+
+    if (log.isTraceEnabled()) {
+      log.trace("tid={} Fetching list of tables...", Thread.currentThread().getId());
+      timer = new OpTimer().start();
+    }
+
     TreeSet<String> tableNames = new TreeSet<String>(Tables.getNameToIdMap(context.getInstance()).keySet());
-    opTimer.stop("Fetched " + tableNames.size() + " table names in %DURATION%");
+
+    if (timer != null) {
+      timer.stop();
+      log.trace("tid={} Fetched {} table names in {}", Thread.currentThread().getId(), tableNames.size(),
+          String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+    }
+
     return tableNames;
   }
 
@@ -144,9 +157,20 @@ public class TableOperationsImpl extends TableOperationsHelper {
     if (tableName.equals(MetadataTable.NAME) || tableName.equals(RootTable.NAME))
       return true;
 
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Checking if table " + tableName + " exists...");
+    OpTimer timer = null;
+
+    if (log.isTraceEnabled()) {
+      log.trace("tid={} Checking if table {} exists...", Thread.currentThread().getId(), tableName);
+      timer = new OpTimer().start();
+    }
+
     boolean exists = Tables.getNameToIdMap(context.getInstance()).containsKey(tableName);
-    opTimer.stop("Checked existance of " + exists + " in %DURATION%");
+
+    if (timer != null) {
+      timer.stop();
+      log.trace("tid={} Checked existance of {} in {}", Thread.currentThread().getId(), exists, String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+    }
+
     return exists;
   }
 
@@ -308,7 +332,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
         try {
           finishFateOperation(opid);
         } catch (Exception e) {
-          log.warn(e.getMessage(), e);
+          log.warn("Exception thrown while finishing fate table operation", e);
         }
     }
   }
@@ -439,17 +463,24 @@ public class TableOperationsImpl extends TableOperationsHelper {
         try {
           TabletClientService.Client client = ThriftUtil.getTServerClient(address, context);
           try {
-            OpTimer opTimer = null;
-            if (log.isTraceEnabled())
-              opTimer = new OpTimer(log, Level.TRACE).start("Splitting tablet " + tl.tablet_extent + " on " + address + " at " + split);
+
+            OpTimer timer = null;
+
+            if (log.isTraceEnabled()) {
+              log.trace("tid={} Splitting tablet {} on {} at {}", Thread.currentThread().getId(), tl.tablet_extent, address, split);
+              timer = new OpTimer().start();
+            }
 
             client.splitTablet(Tracer.traceInfo(), context.rpcCreds(), tl.tablet_extent.toThrift(), TextUtil.getByteBuffer(split));
 
             // just split it, might as well invalidate it in the cache
             tabLocator.invalidateCache(tl.tablet_extent);
 
-            if (opTimer != null)
-              opTimer.stop("Split tablet in %DURATION%");
+            if (timer != null) {
+              timer.stop();
+              log.trace("Split tablet in {}", String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+            }
+
           } finally {
             ThriftUtil.returnClient(client);
           }
@@ -468,8 +499,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
           // Do not silently spin when we repeatedly fail to get the location for a tablet
           locationFailures++;
           if (5 == locationFailures || 0 == locationFailures % 50) {
-            log.warn("Having difficulty locating hosting tabletserver for split " + split + " on table " + tableName + ". Seen " + locationFailures
-                + " failures.");
+            log.warn("Having difficulty locating hosting tabletserver for split {} on table {}. Seen {} failures.", split, tableName, locationFailures);
           }
 
           tabLocator.invalidateCache(tl.tablet_extent);
@@ -542,7 +572,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
           throw (AccumuloSecurityException) e.getCause();
         }
 
-        log.info(e.getMessage() + " ... retrying ...");
+        log.info("{} ... retrying ...", e.getMessage());
         sleepUninterruptibly(3, TimeUnit.SECONDS);
       }
     }
@@ -794,7 +824,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
         case TABLE_DOESNT_EXIST:
           throw new TableNotFoundException(tableId, null, e.getMessage(), e);
         default:
-          log.debug("flush security exception on table id " + tableId);
+          log.debug("flush security exception on table id {}", tableId);
           throw new AccumuloSecurityException(e.user, e.code, e);
       }
     } catch (ThriftTableOperationException e) {
@@ -1154,8 +1184,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
           waitTime = waitFor * 10;
         waitTime = Math.max(100, waitTime);
         waitTime = Math.min(5000, waitTime);
-        log.trace("Waiting for " + waitFor + "(" + maxPerServer + ") tablets, startRow = " + startRow + " lastRow = " + lastRow + ", holes=" + holes
-            + " sleeping:" + waitTime + "ms");
+        log.trace("Waiting for {}({}) tablets, startRow = {} lastRow = {}, holes={} sleeping:{}ms", waitFor, maxPerServer, startRow, lastRow, holes, waitTime);
         sleepUninterruptibly(waitTime, TimeUnit.MILLISECONDS);
       } else {
         break;
@@ -1271,7 +1300,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
         if (pair == null) {
           log.debug("Disk usage request failed.  Pair is null.  Retrying request...", e);
         } else {
-          log.debug("Disk usage request failed " + pair.getFirst() + ", retrying ... ", e);
+          log.debug("Disk usage request failed {}, retrying ... ", pair.getFirst(), e);
         }
         sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
       } catch (TException e) {
@@ -1337,13 +1366,13 @@ public class TableOperationsImpl extends TableOperationsHelper {
 
       for (Entry<String,String> entry : props.entrySet()) {
         if (Property.isClassProperty(entry.getKey()) && !entry.getValue().contains(Constants.CORE_PACKAGE_NAME)) {
-          Logger.getLogger(this.getClass()).info(
-              "Imported table sets '" + entry.getKey() + "' to '" + entry.getValue() + "'.  Ensure this class is on Accumulo classpath.");
+          LoggerFactory.getLogger(this.getClass()).info("Imported table sets '{}' to '{}'.  Ensure this class is on Accumulo classpath.", entry.getKey(),
+              entry.getValue());
         }
       }
 
     } catch (IOException ioe) {
-      Logger.getLogger(this.getClass()).warn("Failed to check if imported table references external java classes : " + ioe.getMessage());
+      LoggerFactory.getLogger(this.getClass()).warn("Failed to check if imported table references external java classes : {}", ioe.getMessage());
     }
 
     List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(UTF_8)), ByteBuffer.wrap(importDir.getBytes(UTF_8)));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6977629e/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
index b2db827..16ef8e8 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java
@@ -48,14 +48,14 @@ import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableComparator;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import static com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly;
 
 public class TabletLocatorImpl extends TabletLocator {
 
-  private static final Logger log = Logger.getLogger(TabletLocatorImpl.class);
+  private static final Logger log = LoggerFactory.getLogger(TabletLocatorImpl.class);
 
   // there seems to be a bug in TreeMap.tailMap related to
   // putting null in the treemap.. therefore instead of
@@ -144,7 +144,7 @@ public class TabletLocatorImpl extends TabletLocator {
       }
 
       if (log.isTraceEnabled())
-        log.trace("Tablet server " + tl.tablet_location + " " + tl.tablet_session + " no longer holds its lock");
+        log.trace("Tablet server {} {} no longer holds its lock", tl.tablet_location, tl.tablet_session);
 
       invalidLocks.add(lock);
 
@@ -166,9 +166,12 @@ public class TabletLocatorImpl extends TabletLocator {
   public <T extends Mutation> void binMutations(ClientContext context, List<T> mutations, Map<String,TabletServerMutations<T>> binnedMutations, List<T> failures)
       throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
 
-    OpTimer opTimer = null;
-    if (log.isTraceEnabled())
-      opTimer = new OpTimer(log, Level.TRACE).start("Binning " + mutations.size() + " mutations for table " + tableId);
+    OpTimer timer = null;
+
+    if (log.isTraceEnabled()) {
+      log.trace("tid={} Binning {} mutations for table {}", Thread.currentThread().getId(), mutations.size(), tableId);
+      timer = new OpTimer().start();
+    }
 
     ArrayList<T> notInCache = new ArrayList<T>();
     Text row = new Text();
@@ -228,8 +231,12 @@ public class TabletLocatorImpl extends TabletLocator {
       }
     }
 
-    if (opTimer != null)
-      opTimer.stop("Binned " + mutations.size() + " mutations for table " + tableId + " to " + binnedMutations.size() + " tservers in %DURATION%");
+    if (timer != null) {
+      timer.stop();
+      log.trace("tid={} Binned {} mutations for table {} to {} tservers in {}", Thread.currentThread().getId(), mutations.size(), tableId,
+          binnedMutations.size(), String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+    }
+
   }
 
   private <T extends Mutation> boolean addMutation(Map<String,TabletServerMutations<T>> binnedMutations, T mutation, TabletLocation tl,
@@ -326,9 +333,12 @@ public class TabletLocatorImpl extends TabletLocator {
      * should not log.
      */
 
-    OpTimer opTimer = null;
-    if (log.isTraceEnabled())
-      opTimer = new OpTimer(log, Level.TRACE).start("Binning " + ranges.size() + " ranges for table " + tableId);
+    OpTimer timer = null;
+
+    if (log.isTraceEnabled()) {
+      log.trace("tid={} Binning {} ranges for table {}", Thread.currentThread().getId(), ranges.size(), tableId);
+      timer = new OpTimer().start();
+    }
 
     LockCheckerSession lcSession = new LockCheckerSession();
 
@@ -360,8 +370,11 @@ public class TabletLocatorImpl extends TabletLocator {
       }
     }
 
-    if (opTimer != null)
-      opTimer.stop("Binned " + ranges.size() + " ranges for table " + tableId + " to " + binnedRanges.size() + " tservers in %DURATION%");
+    if (timer != null) {
+      timer.stop();
+      log.trace("tid={} Binned {} ranges for table {} to {} tservers in {}", Thread.currentThread().getId(), ranges.size(), tableId, binnedRanges.size(),
+          String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+    }
 
     return failures;
   }
@@ -375,7 +388,7 @@ public class TabletLocatorImpl extends TabletLocator {
       wLock.unlock();
     }
     if (log.isTraceEnabled())
-      log.trace("Invalidated extent=" + failedExtent);
+      log.trace("Invalidated extent={}", failedExtent);
   }
 
   @Override
@@ -387,7 +400,7 @@ public class TabletLocatorImpl extends TabletLocator {
       wLock.unlock();
     }
     if (log.isTraceEnabled())
-      log.trace("Invalidated " + keySet.size() + " cache entries for table " + tableId);
+      log.trace("Invalidated {} cache entries for table {}", keySet.size(), tableId);
   }
 
   @Override
@@ -408,7 +421,7 @@ public class TabletLocatorImpl extends TabletLocator {
     lockChecker.invalidateCache(server);
 
     if (log.isTraceEnabled())
-      log.trace("invalidated " + invalidatedCount + " cache entries  table=" + tableId + " server=" + server);
+      log.trace("invalidated {} cache entries  table={} server={}", invalidatedCount, tableId, server);
 
   }
 
@@ -423,17 +436,19 @@ public class TabletLocatorImpl extends TabletLocator {
       wLock.unlock();
     }
     if (log.isTraceEnabled())
-      log.trace("invalidated all " + invalidatedCount + " cache entries for table=" + tableId);
+      log.trace("invalidated all {} cache entries for table={}", invalidatedCount, tableId);
   }
 
   @Override
   public TabletLocation locateTablet(ClientContext context, Text row, boolean skipRow, boolean retry) throws AccumuloException, AccumuloSecurityException,
       TableNotFoundException {
 
-    OpTimer opTimer = null;
-    if (log.isTraceEnabled())
-      opTimer = new OpTimer(log, Level.TRACE).start("Locating tablet  table=" + tableId + " row=" + TextUtil.truncate(row) + "  skipRow=" + skipRow + " retry="
-          + retry);
+    OpTimer timer = null;
+
+    if (log.isTraceEnabled()) {
+      log.trace("tid={} Locating tablet  table={} row={} skipRow={} retry={}", Thread.currentThread().getId(), tableId, TextUtil.truncate(row), skipRow, retry);
+      timer = new OpTimer().start();
+    }
 
     while (true) {
 
@@ -443,12 +458,15 @@ public class TabletLocatorImpl extends TabletLocator {
       if (retry && tl == null) {
         sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
         if (log.isTraceEnabled())
-          log.trace("Failed to locate tablet containing row " + TextUtil.truncate(row) + " in table " + tableId + ", will retry...");
+          log.trace("Failed to locate tablet containing row {} in table {}, will retry...", TextUtil.truncate(row), tableId);
         continue;
       }
 
-      if (opTimer != null)
-        opTimer.stop("Located tablet " + (tl == null ? null : tl.tablet_extent) + " at " + (tl == null ? null : tl.tablet_location) + " in %DURATION%");
+      if (timer != null) {
+        timer.stop();
+        log.trace("tid={} Located tablet {} at {} in {}", Thread.currentThread().getId(), (tl == null ? "null" : tl.tablet_extent), (tl == null ? "null"
+            : tl.tablet_location), String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+      }
 
       return tl;
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6977629e/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
index f263581..0b09808 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
@@ -34,6 +34,7 @@ import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -63,18 +64,18 @@ import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.hadoop.io.Text;
 import org.apache.htrace.wrappers.TraceRunnable;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
 import org.apache.thrift.TApplicationException;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransport;
 import org.apache.thrift.transport.TTransportException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.net.HostAndPort;
 
 public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value>> {
 
-  private static final Logger log = Logger.getLogger(TabletServerBatchReaderIterator.class);
+  private static final Logger log = LoggerFactory.getLogger(TabletServerBatchReaderIterator.class);
 
   private final ClientContext context;
   private final Instance instance;
@@ -246,7 +247,8 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
         lastFailureSize = failures.size();
 
         if (log.isTraceEnabled())
-          log.trace("Failed to bin " + failures.size() + " ranges, tablet locations were null, retrying in 100ms");
+          log.trace("Failed to bin {} ranges, tablet locations were null, retrying in 100ms", failures.size());
+
         try {
           Thread.sleep(100);
         } catch (InterruptedException e) {
@@ -280,7 +282,7 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
   private void processFailures(Map<KeyExtent,List<Range>> failures, ResultReceiver receiver, List<Column> columns) throws AccumuloException,
       AccumuloSecurityException, TableNotFoundException {
     if (log.isTraceEnabled())
-      log.trace("Failed to execute multiscans against " + failures.size() + " tablets, retrying...");
+      log.trace("Failed to execute multiscans against {} tablets, retrying...", failures.size());
 
     try {
       Thread.sleep(failSleepTime);
@@ -363,10 +365,10 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
 
           locator.invalidateCache(context.getInstance(), tsLocation);
         }
-        log.debug(e.getMessage(), e);
+        log.debug("IOException thrown", e);
       } catch (AccumuloSecurityException e) {
         e.setTableInfo(getTableInfo());
-        log.debug(e.getMessage(), e);
+        log.debug("AccumuloSecurityException thrown", e);
 
         Tables.clearCache(instance);
         if (!Tables.exists(instance, table))
@@ -375,9 +377,9 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
           fatalException = e;
       } catch (Throwable t) {
         if (queryThreadPool.isShutdown())
-          log.debug(t.getMessage(), t);
+          log.debug("Caught exception, but queryThreadPool is shutdown", t);
         else
-          log.warn(t.getMessage(), t);
+          log.warn("Caught exception, but queryThreadPool is not shutdown", t);
         fatalException = t;
       } finally {
         semaphore.release();
@@ -389,17 +391,17 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
             try {
               processFailures(failures, receiver, columns);
             } catch (TableNotFoundException e) {
-              log.debug(e.getMessage(), e);
+              log.debug("{}", e.getMessage(), e);
               fatalException = e;
             } catch (AccumuloException e) {
-              log.debug(e.getMessage(), e);
+              log.debug("{}", e.getMessage(), e);
               fatalException = e;
             } catch (AccumuloSecurityException e) {
               e.setTableInfo(getTableInfo());
-              log.debug(e.getMessage(), e);
+              log.debug("{}", e.getMessage(), e);
               fatalException = e;
             } catch (Throwable t) {
-              log.debug(t.getMessage(), t);
+              log.debug("{}", t.getMessage(), t);
               fatalException = t;
             }
 
@@ -625,8 +627,14 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
 
       try {
 
-        OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Starting multi scan, tserver=" + server + "  #tablets=" + requested.size() + "  #ranges="
-            + sumSizes(requested.values()) + " ssil=" + options.serverSideIteratorList + " ssio=" + options.serverSideIteratorOptions);
+        OpTimer timer = null;
+
+        if (log.isTraceEnabled()) {
+          log.trace("tid={} Starting multi scan, tserver={}  #tablets={}  #ranges={} ssil={} ssio={}", Thread.currentThread().getId(), server,
+              requested.size(), sumSizes(requested.values()), options.serverSideIteratorList, options.serverSideIteratorOptions);
+
+          timer = new OpTimer().start();
+        }
 
         TabletType ttype = TabletType.type(requested.keySet());
         boolean waitForWrites = !ThriftScanner.serversWaitedForWrites.get(ttype).contains(server);
@@ -641,8 +649,11 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
 
         MultiScanResult scanResult = imsr.result;
 
-        opTimer.stop("Got 1st multi scan results, #results=" + scanResult.results.size() + (scanResult.more ? "  scanID=" + imsr.scanID : "")
-            + " in %DURATION%");
+        if (timer != null) {
+          timer.stop();
+          log.trace("tid={} Got 1st multi scan results, #results={} {} in {}", Thread.currentThread().getId(), scanResult.results.size(),
+              (scanResult.more ? "scanID=" + imsr.scanID : ""), String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+        }
 
         ArrayList<Entry<Key,Value>> entries = new ArrayList<Map.Entry<Key,Value>>(scanResult.results.size());
         for (TKeyValue kv : scanResult.results) {
@@ -657,14 +668,24 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
 
         trackScanning(failures, unscanned, scanResult);
 
+        AtomicLong nextOpid = new AtomicLong();
+
         while (scanResult.more) {
 
           timeoutTracker.check();
 
-          opTimer.start("Continuing multi scan, scanid=" + imsr.scanID);
+          if (timer != null) {
+            log.trace("tid={} oid={} Continuing multi scan, scanid={}", Thread.currentThread().getId(), nextOpid.get(), imsr.scanID);
+            timer.reset().start();
+          }
+
           scanResult = client.continueMultiScan(Tracer.traceInfo(), imsr.scanID);
-          opTimer.stop("Got more multi scan results, #results=" + scanResult.results.size() + (scanResult.more ? "  scanID=" + imsr.scanID : "")
-              + " in %DURATION%");
+
+          if (timer != null) {
+            timer.stop();
+            log.trace("tid={} oid={} Got more multi scan results, #results={} {} in {}", Thread.currentThread().getId(), nextOpid.getAndIncrement(),
+                scanResult.results.size(), (scanResult.more ? " scanID=" + imsr.scanID : ""), String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+          }
 
           entries = new ArrayList<Map.Entry<Key,Value>>(scanResult.results.size());
           for (TKeyValue kv : scanResult.results) {
@@ -686,20 +707,20 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
         ThriftUtil.returnClient(client);
       }
     } catch (TTransportException e) {
-      log.debug("Server : " + server + " msg : " + e.getMessage());
+      log.debug("Server : {} msg : {}", server, e.getMessage());
       timeoutTracker.errorOccured(e);
       throw new IOException(e);
     } catch (ThriftSecurityException e) {
-      log.debug("Server : " + server + " msg : " + e.getMessage(), e);
+      log.debug("Server : {} msg : {}", server, e.getMessage(), e);
       throw new AccumuloSecurityException(e.user, e.code, e);
     } catch (TApplicationException e) {
-      log.debug("Server : " + server + " msg : " + e.getMessage(), e);
+      log.debug("Server : {} msg : {}", server, e.getMessage(), e);
       throw new AccumuloServerException(server, e);
     } catch (NoSuchScanIDException e) {
-      log.debug("Server : " + server + " msg : " + e.getMessage(), e);
+      log.debug("Server : {} msg : {}", server, e.getMessage(), e);
       throw new IOException(e);
     } catch (TException e) {
-      log.debug("Server : " + server + " msg : " + e.getMessage(), e);
+      log.debug("Server : {} msg : {}", server, e.getMessage(), e);
       timeoutTracker.errorOccured(e);
       throw new IOException(e);
     } finally {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6977629e/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
index 5437cc4..c2cc1e3 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java
@@ -26,6 +26,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.SortedSet;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
@@ -60,15 +61,15 @@ import org.apache.accumulo.core.trace.Tracer;
 import org.apache.accumulo.core.trace.thrift.TInfo;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
 import org.apache.thrift.TApplicationException;
 import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.net.HostAndPort;
 
 public class ThriftScanner {
-  private static final Logger log = Logger.getLogger(ThriftScanner.class);
+  private static final Logger log = LoggerFactory.getLogger(ThriftScanner.class);
 
   public static final Map<TabletType,Set<String>> serversWaitedForWrites = new EnumMap<TabletType,Set<String>>(TabletType.class);
 
@@ -115,12 +116,12 @@ public class ThriftScanner {
     } catch (TApplicationException tae) {
       throw new AccumuloServerException(server, tae);
     } catch (TooManyFilesException e) {
-      log.debug("Tablet (" + extent + ") has too many files " + server + " : " + e);
+      log.debug("Tablet ({}) has too many files {} : {}", extent, server, e.getMessage());
     } catch (ThriftSecurityException e) {
-      log.warn("Security Violation in scan request to " + server + ": " + e);
+      log.warn("Security Violation in scan request to {}: {}", server, e.getMessage());
       throw new AccumuloSecurityException(e.user, e.code, e);
     } catch (TException e) {
-      log.debug("Error getting transport to " + server + " : " + e);
+      log.debug("Error getting transport to {}: {}", server, e.getMessage());
     }
 
     throw new AccumuloException("getBatchFromServer: failed");
@@ -156,7 +157,7 @@ public class ThriftScanner {
         List<IterInfo> serverSideIteratorList, Map<String,Map<String,String>> serverSideIteratorOptions, boolean isolated, long readaheadThreshold,
         long batchTimeOut) {
       this.context = context;
-      ;
+
       this.authorizations = authorizations;
 
       columns = new ArrayList<Column>(fetchedColumns.size());
@@ -237,9 +238,9 @@ public class ThriftScanner {
 
               error = "Failed to locate tablet for table : " + scanState.tableId + " row : " + scanState.startRow;
               if (!error.equals(lastError))
-                log.debug(error);
+                log.debug("{}", error);
               else if (log.isTraceEnabled())
-                log.trace(error);
+                log.trace("{}", error);
               lastError = error;
               sleepMillis = pause(sleepMillis);
             } else {
@@ -259,14 +260,14 @@ public class ThriftScanner {
               }
             }
           } catch (AccumuloServerException e) {
-            log.debug("Scan failed, server side exception : " + e.getMessage());
+            log.debug("Scan failed, server side exception : {}", e.getMessage());
             throw e;
           } catch (AccumuloException e) {
             error = "exception from tablet loc " + e.getMessage();
             if (!error.equals(lastError))
-              log.debug(error);
+              log.debug("{}", error);
             else if (log.isTraceEnabled())
-              log.trace(error);
+              log.trace("{}", error);
 
             lastError = error;
             sleepMillis = pause(sleepMillis);
@@ -290,9 +291,9 @@ public class ThriftScanner {
         } catch (NotServingTabletException e) {
           error = "Scan failed, not serving tablet " + loc;
           if (!error.equals(lastError))
-            log.debug(error);
+            log.debug("{}", error);
           else if (log.isTraceEnabled())
-            log.trace(error);
+            log.trace("{}", error);
           lastError = error;
 
           TabletLocator.getLocator(context, scanState.tableId).invalidateCache(loc.tablet_extent);
@@ -308,9 +309,9 @@ public class ThriftScanner {
         } catch (NoSuchScanIDException e) {
           error = "Scan failed, no such scan id " + scanState.scanID + " " + loc;
           if (!error.equals(lastError))
-            log.debug(error);
+            log.debug("{}", error);
           else if (log.isTraceEnabled())
-            log.trace(error);
+            log.trace("{}", error);
           lastError = error;
 
           if (scanState.isolated)
@@ -320,14 +321,14 @@ public class ThriftScanner {
         } catch (TooManyFilesException e) {
           error = "Tablet has too many files " + loc + " retrying...";
           if (!error.equals(lastError)) {
-            log.debug(error);
+            log.debug("{}", error);
             tooManyFilesCount = 0;
           } else {
             tooManyFilesCount++;
             if (tooManyFilesCount == 300)
-              log.warn(error);
+              log.warn("{}", error);
             else if (log.isTraceEnabled())
-              log.trace(error);
+              log.trace("{}", error);
           }
           lastError = error;
 
@@ -344,9 +345,9 @@ public class ThriftScanner {
           TabletLocator.getLocator(context, scanState.tableId).invalidateCache(context.getInstance(), loc.tablet_location);
           error = "Scan failed, thrift error " + e.getClass().getName() + "  " + e.getMessage() + " " + loc;
           if (!error.equals(lastError))
-            log.debug(error);
+            log.debug("{}", error);
           else if (log.isTraceEnabled())
-            log.trace(error);
+            log.trace("{}", error);
           lastError = error;
           loc = null;
 
@@ -380,7 +381,7 @@ public class ThriftScanner {
     if (scanState.finished)
       return null;
 
-    OpTimer opTimer = new OpTimer(log, Level.TRACE);
+    OpTimer timer = null;
 
     final TInfo tinfo = Tracer.traceInfo();
     final HostAndPort parsedLocation = HostAndPort.fromString(loc.tablet_location);
@@ -399,7 +400,11 @@ public class ThriftScanner {
         String msg = "Starting scan tserver=" + loc.tablet_location + " tablet=" + loc.tablet_extent + " range=" + scanState.range + " ssil="
             + scanState.serverSideIteratorList + " ssio=" + scanState.serverSideIteratorOptions;
         Thread.currentThread().setName(msg);
-        opTimer.start(msg);
+
+        if (log.isTraceEnabled()) {
+          log.trace("tid={} {}", Thread.currentThread().getId(), msg);
+          timer = new OpTimer().start();
+        }
 
         TabletType ttype = TabletType.type(loc.tablet_extent);
         boolean waitForWrites = !serversWaitedForWrites.get(ttype).contains(loc.tablet_location);
@@ -420,7 +425,11 @@ public class ThriftScanner {
         // log.debug("Calling continue scan : "+scanState.range+"  loc = "+loc);
         String msg = "Continuing scan tserver=" + loc.tablet_location + " scanid=" + scanState.scanID;
         Thread.currentThread().setName(msg);
-        opTimer.start(msg);
+
+        if (log.isTraceEnabled()) {
+          log.trace("tid={} {}", Thread.currentThread().getId(), msg);
+          timer = new OpTimer().start();
+        }
 
         sr = client.continueScan(tinfo, scanState.scanID);
         if (!sr.more) {
@@ -433,17 +442,36 @@ public class ThriftScanner {
         // log.debug("No more : tab end row = "+loc.tablet_extent.getEndRow()+" range = "+scanState.range);
         if (loc.tablet_extent.getEndRow() == null) {
           scanState.finished = true;
-          opTimer.stop("Completely finished scan in %DURATION% #results=" + sr.results.size());
+
+          if (timer != null) {
+            timer.stop();
+            log.trace("tid={} Completely finished scan in {} #results={}", Thread.currentThread().getId(),
+                String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)), sr.results.size());
+          }
+
         } else if (scanState.range.getEndKey() == null || !scanState.range.afterEndKey(new Key(loc.tablet_extent.getEndRow()).followingKey(PartialKey.ROW))) {
           scanState.startRow = loc.tablet_extent.getEndRow();
           scanState.skipStartRow = true;
-          opTimer.stop("Finished scanning tablet in %DURATION% #results=" + sr.results.size());
+
+          if (timer != null) {
+            timer.stop();
+            log.trace("tid={} Finished scanning tablet in {} #results={}", Thread.currentThread().getId(),
+                String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)), sr.results.size());
+          }
         } else {
           scanState.finished = true;
-          opTimer.stop("Completely finished scan in %DURATION% #results=" + sr.results.size());
+          if (timer != null) {
+            timer.stop();
+            log.trace("tid={} Completely finished in {} #results={}", Thread.currentThread().getId(),
+                String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)), sr.results.size());
+          }
         }
       } else {
-        opTimer.stop("Finished scan in %DURATION% #results=" + sr.results.size() + " scanid=" + scanState.scanID);
+        if (timer != null) {
+          timer.stop();
+          log.trace("tid={} Finished scan in {} #results={} scanid={}", Thread.currentThread().getId(),
+              String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)), sr.results.size(), scanState.scanID);
+        }
       }
 
       Key.decompress(sr.results);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6977629e/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
index 0d294b8..42dd881 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
@@ -27,6 +27,7 @@ import java.util.SortedMap;
 import java.util.SortedSet;
 import java.util.TreeMap;
 import java.util.TreeSet;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
@@ -56,11 +57,12 @@ import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.TextUtil;
 import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class MetadataLocationObtainer implements TabletLocationObtainer {
-  private static final Logger log = Logger.getLogger(MetadataLocationObtainer.class);
+  private static final Logger log = LoggerFactory.getLogger(MetadataLocationObtainer.class);
+
   private SortedSet<Column> locCols;
   private ArrayList<Column> columns;
 
@@ -77,10 +79,14 @@ public class MetadataLocationObtainer implements TabletLocationObtainer {
       throws AccumuloSecurityException, AccumuloException {
 
     try {
-      OpTimer opTimer = null;
-      if (log.isTraceEnabled())
-        opTimer = new OpTimer(log, Level.TRACE).start("Looking up in " + src.tablet_extent.getTableId() + " row=" + TextUtil.truncate(row) + "  extent="
-            + src.tablet_extent + " tserver=" + src.tablet_location);
+
+      OpTimer timer = null;
+
+      if (log.isTraceEnabled()) {
+        log.trace("tid={} Looking up in {} row={} extent={} tserver={}", Thread.currentThread().getId(), src.tablet_extent.getTableId(),
+            TextUtil.truncate(row), src.tablet_extent, src.tablet_location);
+        timer = new OpTimer().start();
+      }
 
       Range range = new Range(row, true, stopRow, true);
 
@@ -106,8 +112,11 @@ public class MetadataLocationObtainer implements TabletLocationObtainer {
         decodeRows(encodedResults, results);
       }
 
-      if (opTimer != null)
-        opTimer.stop("Got " + results.size() + " results  from " + src.tablet_extent + " in %DURATION%");
+      if (timer != null) {
+        timer.stop();
+        log.trace("tid={} Got {} results from {} in {}", Thread.currentThread().getId(), results.size(), src.tablet_extent,
+            String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+      }
 
       // if (log.isTraceEnabled()) log.trace("results "+results);
 
@@ -115,15 +124,15 @@ public class MetadataLocationObtainer implements TabletLocationObtainer {
 
     } catch (AccumuloServerException ase) {
       if (log.isTraceEnabled())
-        log.trace(src.tablet_extent.getTableId() + " lookup failed, " + src.tablet_location + " server side exception");
+        log.trace("{} lookup failed, {} server side exception", src.tablet_extent.getTableId(), src.tablet_location);
       throw ase;
     } catch (NotServingTabletException e) {
       if (log.isTraceEnabled())
-        log.trace(src.tablet_extent.getTableId() + " lookup failed, " + src.tablet_location + " not serving " + src.tablet_extent);
+        log.trace("{} lookup failed, {} not serving {}", src.tablet_extent.getTableId(), src.tablet_location, src.tablet_extent);
       parent.invalidateCache(src.tablet_extent);
     } catch (AccumuloException e) {
       if (log.isTraceEnabled())
-        log.trace(src.tablet_extent.getTableId() + " lookup failed", e);
+        log.trace("{} lookup failed", src.tablet_extent.getTableId(), e);
       parent.invalidateCache(context.getInstance(), src.tablet_location);
     }
 
@@ -176,14 +185,14 @@ public class MetadataLocationObtainer implements TabletLocationObtainer {
       if (failures.size() > 0) {
         // invalidate extents in parents cache
         if (log.isTraceEnabled())
-          log.trace("lookupTablets failed for " + failures.size() + " extents");
+          log.trace("lookupTablets failed for {} extents", failures.size());
         parent.invalidateCache(failures.keySet());
       }
     } catch (IOException e) {
-      log.trace("lookupTablets failed server=" + tserver, e);
+      log.trace("lookupTablets failed server={}", tserver, e);
       parent.invalidateCache(context.getInstance(), tserver);
     } catch (AccumuloServerException e) {
-      log.trace("lookupTablets failed server=" + tserver, e);
+      log.trace("lookupTablets failed server={}", tserver, e);
       throw e;
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6977629e/core/src/main/java/org/apache/accumulo/core/util/OpTimer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/OpTimer.java b/core/src/main/java/org/apache/accumulo/core/util/OpTimer.java
index 564a824..0ba7f8f 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/OpTimer.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/OpTimer.java
@@ -16,37 +16,126 @@
  */
 package org.apache.accumulo.core.util;
 
-import java.util.concurrent.atomic.AtomicLong;
+import java.io.Closeable;
+import java.util.concurrent.TimeUnit;
 
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
+/**
+ * Provides a stop watch for timing a single type of event. This code is based on the org.apache.hadoop.util.StopWatch available in hadoop 2.7.0
+ */
+public class OpTimer implements Closeable {
+
+  private boolean isStarted;
+  private long startNanos;
+  private long currentElapsedNanos;
 
-public class OpTimer {
-  private Logger log;
-  private Level level;
-  private long t1;
-  private long opid;
-  private static AtomicLong nextOpid = new AtomicLong();
+  /**
+   * Create an OpTimer instance. The timer is not running.
+   */
+  public OpTimer() {}
 
-  public OpTimer(Logger log, Level level) {
-    this.log = log;
-    this.level = level;
+  /**
+   * Returns timer running state
+   *
+   * @return true if timer is running
+   */
+  public boolean isRunning() {
+    return isStarted;
+  }
+
+  /**
+   * Start the timer instance.
+   *
+   * @return this instance for fluent chaining.
+   * @throws IllegalStateException
+   *           if start is called on running instance.
+   */
+  public OpTimer start() throws IllegalStateException {
+    if (isStarted) {
+      throw new IllegalStateException("OpTimer is already running");
+    }
+    isStarted = true;
+    startNanos = System.nanoTime();
+    return this;
+  }
+
+  /**
+   * Stop the timer instance.
+   *
+   * @return this instance for fluent chaining.
+   * @throws IllegalStateException
+   *           if stop is called on instance that is not running.
+   */
+  public OpTimer stop() throws IllegalStateException {
+    if (!isStarted) {
+      throw new IllegalStateException("OpTimer is already stopped");
+    }
+    long now = System.nanoTime();
+    isStarted = false;
+    currentElapsedNanos += now - startNanos;
+    return this;
   }
 
-  public OpTimer start(String msg) {
-    opid = nextOpid.getAndIncrement();
-    if (log.isEnabledFor(level))
-      log.log(level, "tid=" + Thread.currentThread().getId() + " oid=" + opid + "  " + msg);
-    t1 = System.currentTimeMillis();
+  /**
+   * Stops timer instance and current elapsed time to 0.
+   *
+   * @return this instance for fluent chaining
+   */
+  public OpTimer reset() {
+    currentElapsedNanos = 0;
+    isStarted = false;
     return this;
   }
 
-  public void stop(String msg) {
-    if (log.isEnabledFor(level)) {
-      long t2 = System.currentTimeMillis();
-      String duration = String.format("%.3f secs", (t2 - t1) / 1000.0);
-      msg = msg.replace("%DURATION%", duration);
-      log.log(level, "tid=" + Thread.currentThread().getId() + " oid=" + opid + "  " + msg);
+  /**
+   * Converts current timer value to specific unit. The conversion to courser granularities truncate with loss of precision.
+   *
+   * @param timeUnit
+   *          the time unit that will converted to.
+   * @return truncated time in unit of specified time unit.
+   */
+  public long now(TimeUnit timeUnit) {
+    return timeUnit.convert(now(), TimeUnit.NANOSECONDS);
+  }
+
+  /**
+   * Returns the current elapsed time scaled to the provided time unit. This method does not truncate like {@link #now(TimeUnit)} but returns the value as a
+   * double. </p> Note: this method is not included in the hadoop 2.7 org.apache.hadoop.util.StopWatch class. If that class is adopted, then provisions will be
+   * required to replace this method.
+   *
+   * @param timeUnit
+   *          the time unit to scale the elapsed time to.
+   * @return the elapsed time of this instance scaled to the provided time unit.
+   */
+  public double scale(TimeUnit timeUnit) {
+    return (double) now() / TimeUnit.NANOSECONDS.convert(1L, timeUnit);
+  }
+
+  /**
+   * Returns current timer elapsed time as nanoseconds.
+   *
+   * @return elapsed time in nanoseconds.
+   */
+  public long now() {
+    return isStarted ? System.nanoTime() - startNanos + currentElapsedNanos : currentElapsedNanos;
+  }
+
+  /**
+   * Return the current elapsed time in nanoseconds as a string.
+   *
+   * @return timer elapsed time as nanoseconds.
+   */
+  @Override
+  public String toString() {
+    return String.valueOf(now());
+  }
+
+  /**
+   * If the timer is running, stop it. This method can be called even if the timer is not running.
+   */
+  @Override
+  public void close() {
+    if (isStarted) {
+      stop();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6977629e/core/src/test/java/org/apache/accumulo/core/util/OpTimerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/util/OpTimerTest.java b/core/src/test/java/org/apache/accumulo/core/util/OpTimerTest.java
new file mode 100644
index 0000000..af87bf1
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/util/OpTimerTest.java
@@ -0,0 +1,203 @@
+/*
+ * 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.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Exercise basic timer (org.apache.hadoop.util.StopWatch) functionality. Current usage requires ability to reset timer.
+ */
+public class OpTimerTest {
+
+  private static Logger log = LoggerFactory.getLogger(OpTimerTest.class);
+
+  /**
+   * Validate reset functionality
+   */
+  @Test
+  public void verifyReset() {
+
+    OpTimer timer = new OpTimer().start();
+
+    try {
+      Thread.sleep(50);
+    } catch (InterruptedException ex) {
+      log.info("sleep sleep interrupted");
+      Thread.currentThread().interrupt();
+    }
+
+    timer.stop();
+
+    long tValue = timer.now();
+
+    log.debug("Time value before reset {}", String.format("%.3f ms", timer.scale(TimeUnit.MILLISECONDS)));
+
+    timer.reset().start();
+
+    try {
+      Thread.sleep(1);
+    } catch (InterruptedException ex) {
+      log.info("sleep sleep interrupted");
+      Thread.currentThread().interrupt();
+    }
+
+    timer.stop();
+
+    assertTrue(timer.now() > 0);
+
+    assertTrue(tValue > timer.now());
+
+    timer.reset();
+
+    log.debug("Time value after reset {}", String.format("%.3f ms", timer.scale(TimeUnit.MILLISECONDS)));
+
+    assertEquals(0, timer.now());
+
+  }
+
+  /**
+   * Verify that IllegalStateException is thrown when calling stop when timer has not been started.
+   */
+  @Test(expected = IllegalStateException.class)
+  public void verifyExceptionCallingStopWhenNotStarted() {
+
+    OpTimer timer = new OpTimer();
+
+    assertFalse(timer.isRunning());
+
+    // should throw exception - not running
+    timer.stop();
+  }
+
+  /**
+   * Verify that IllegalStateException is thrown when calling start on running timer.
+   */
+  @Test(expected = IllegalStateException.class)
+  public void verifyExceptionCallingStartWhenRunning() {
+
+    OpTimer timer = new OpTimer().start();
+
+    try {
+      Thread.sleep(50);
+    } catch (InterruptedException ex) {
+      log.info("sleep sleep interrupted");
+      Thread.currentThread().interrupt();
+    }
+
+    assertTrue(timer.isRunning());
+
+    // should throw exception - already running
+    timer.start();
+  }
+
+  /**
+   * Verify that IllegalStateException is thrown when calling stop when not running.
+   */
+  @Test(expected = IllegalStateException.class)
+  public void verifyExceptionCallingStopWhenNotRunning() {
+
+    OpTimer timer = new OpTimer().start();
+
+    try {
+      Thread.sleep(50);
+    } catch (InterruptedException ex) {
+      log.info("sleep sleep interrupted");
+      Thread.currentThread().interrupt();
+    }
+
+    assertTrue(timer.isRunning());
+
+    timer.stop();
+
+    assertFalse(timer.isRunning());
+
+    // should throw exception
+    timer.stop();
+  }
+
+  /**
+   * Validate that start / stop accumulates time.
+   */
+  @Test
+  public void verifyElapsed() {
+
+    OpTimer timer = new OpTimer().start();
+
+    try {
+      Thread.sleep(50);
+    } catch (InterruptedException ex) {
+      log.info("sleep sleep interrupted");
+      Thread.currentThread().interrupt();
+    }
+
+    timer.stop();
+
+    long tValue = timer.now();
+
+    log.debug("Time value after first stop {}", String.format("%.3f ms", timer.scale(TimeUnit.MILLISECONDS)));
+
+    timer.start();
+
+    try {
+      Thread.sleep(10);
+    } catch (InterruptedException ex) {
+      log.info("sleep sleep interrupted");
+      Thread.currentThread().interrupt();
+    }
+
+    timer.stop();
+
+    log.debug("Time value after second stop {}", String.format("%.3f ms", timer.scale(TimeUnit.MILLISECONDS)));
+
+    assertTrue(tValue < timer.now());
+
+  }
+
+  /**
+   * Validate that scale returns correct values.
+   */
+  @Test
+  public void scale() {
+    OpTimer timer = new OpTimer().start();
+
+    try {
+      Thread.sleep(50);
+    } catch (InterruptedException ex) {
+      log.info("sleep sleep interrupted");
+      Thread.currentThread().interrupt();
+    }
+
+    timer.stop();
+
+    long tValue = timer.now();
+
+    double millis = timer.scale(TimeUnit.MILLISECONDS);
+
+    assertEquals(tValue / 1000000.0, timer.scale(TimeUnit.MILLISECONDS), 0.00000001);
+
+    assertEquals(tValue / 1000000000.0, timer.scale(TimeUnit.SECONDS), 0.00000001);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6977629e/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java b/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
index 0d7aaf1..e4e73d2 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java
@@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.AccumuloException;
@@ -52,10 +53,9 @@ import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-
 import com.google.common.base.Joiner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * An implementation of Instance that looks in HDFS and ZooKeeper to find the master and root tablet location.
@@ -78,17 +78,26 @@ public class HdfsZooInstance implements Instance {
 
   private static ZooCache zooCache;
   private static String instanceId = null;
-  private static final Logger log = Logger.getLogger(HdfsZooInstance.class);
+  private static final Logger log = LoggerFactory.getLogger(HdfsZooInstance.class);
 
   @Override
   public String getRootTabletLocation() {
     String zRootLocPath = ZooUtil.getRoot(this) + RootTable.ZROOT_TABLET_LOCATION;
 
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up root tablet location in zoocache.");
+    OpTimer timer = null;
+
+    if (log.isTraceEnabled()) {
+      log.trace("tid={} Looking up root tablet location in zoocache.", Thread.currentThread().getId());
+      timer = new OpTimer().start();
+    }
 
     byte[] loc = zooCache.get(zRootLocPath);
 
-    opTimer.stop("Found root tablet at " + (loc == null ? null : new String(loc, UTF_8)) + " in %DURATION%");
+    if (timer != null) {
+      timer.stop();
+      log.trace("tid={} Found root tablet at {} in {}", Thread.currentThread().getId(), (loc == null ? "null" : new String(loc, UTF_8)),
+          String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+    }
 
     if (loc == null) {
       return null;
@@ -102,11 +111,20 @@ public class HdfsZooInstance implements Instance {
 
     String masterLocPath = ZooUtil.getRoot(this) + Constants.ZMASTER_LOCK;
 
-    OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up master location in zoocache.");
+    OpTimer timer = null;
+
+    if (log.isTraceEnabled()) {
+      log.trace("tid={} Looking up master location in zoocache.", Thread.currentThread().getId());
+      timer = new OpTimer().start();
+    }
 
     byte[] loc = ZooLock.getLockData(zooCache, masterLocPath, null);
 
-    opTimer.stop("Found master at " + (loc == null ? null : new String(loc, UTF_8)) + " in %DURATION%");
+    if (timer != null) {
+      timer.stop();
+      log.trace("tid={} Found master at {} in {}", Thread.currentThread().getId(), (loc == null ? "null" : new String(loc, UTF_8)),
+          String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+    }
 
     if (loc == null) {
       return Collections.emptyList();
@@ -133,7 +151,7 @@ public class HdfsZooInstance implements Instance {
         throw new RuntimeException(e);
       }
       Path instanceIdPath = Accumulo.getAccumuloInstanceIdPath(fs);
-      log.trace("Looking for instanceId from " + instanceIdPath);
+      log.trace("Looking for instanceId from {}", instanceIdPath);
       String instanceIdFromFile = ZooUtil.getInstanceIDFromHdfs(instanceIdPath, acuConf);
       instanceId = instanceIdFromFile;
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6977629e/test/src/main/java/org/apache/accumulo/test/NativeMapStressTest.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/NativeMapStressTest.java b/test/src/main/java/org/apache/accumulo/test/NativeMapStressTest.java
index 72831d8..d527a60 100644
--- a/test/src/main/java/org/apache/accumulo/test/NativeMapStressTest.java
+++ b/test/src/main/java/org/apache/accumulo/test/NativeMapStressTest.java
@@ -25,6 +25,8 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Random;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -32,12 +34,12 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.tserver.NativeMap;
 import org.apache.hadoop.io.Text;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class NativeMapStressTest {
 
-  private static final Logger log = Logger.getLogger(NativeMapStressTest.class);
+  private static final Logger log = LoggerFactory.getLogger(NativeMapStressTest.class);
 
   public static void main(String[] args) {
     testLotsOfMapDeletes(true);
@@ -69,9 +71,13 @@ public class NativeMapStressTest {
 
           Random r = new Random();
 
-          OpTimer opTimer = new OpTimer(log, Level.INFO);
+          OpTimer timer = null;
+          AtomicLong nextOpid = new AtomicLong();
 
-          opTimer.start("Creating map of size " + mapSizePerThread);
+          if (log.isInfoEnabled()) {
+            log.info("tid={} oid={} Creating map of size {}", Thread.currentThread().getId(), nextOpid.get(), mapSizePerThread);
+            timer = new OpTimer().start();
+          }
 
           for (int i = 0; i < mapSizePerThread; i++) {
             String row = String.format("r%08d", i);
@@ -79,9 +85,17 @@ public class NativeMapStressTest {
             put(nm, row, val, i);
           }
 
-          opTimer.stop("Created map of size " + nm.size() + " in %DURATION%");
+          if (timer != null) {
+
+            // stop and log created elapsed time
+            timer.stop();
+            log.info("tid={} oid={} Created map of size {} in {}", Thread.currentThread().getId(), nextOpid.getAndIncrement(), nm.size(),
+                String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
 
-          opTimer.start("Doing " + getsPerThread + " gets()");
+            // start timer for gets
+            log.info("tid={} oid={} Doing {} gets()", Thread.currentThread().getId(), nextOpid.get(), getsPerThread);
+            timer.reset().start();
+          }
 
           for (int i = 0; i < getsPerThread; i++) {
             String row = String.format("r%08d", r.nextInt(mapSizePerThread));
@@ -89,15 +103,23 @@ public class NativeMapStressTest {
 
             Value value = nm.get(new Key(new Text(row)));
             if (value == null || !value.toString().equals(val)) {
-              log.error("nm.get(" + row + ") failed");
+              log.error("nm.get({}) failed", row);
             }
           }
 
-          opTimer.stop("Finished " + getsPerThread + " gets in %DURATION%");
+          if (timer != null) {
 
-          int scanned = 0;
+            // stop and log created elapsed time
+            timer.stop();
+            log.info("tid={} oid={} Finished {} gets in {}", Thread.currentThread().getId(), nextOpid.getAndIncrement(), getsPerThread,
+                String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+
+            // start timer for random iterations
+            log.info("tid={} oid={} Doing {} random iterations", Thread.currentThread().getId(), nextOpid.get(), getsPerThread);
+            timer.reset().start();
+          }
 
-          opTimer.start("Doing " + getsPerThread + " random iterations");
+          int scanned = 0;
 
           for (int i = 0; i < getsPerThread; i++) {
             int startRow = r.nextInt(mapSizePerThread);
@@ -113,7 +135,7 @@ public class NativeMapStressTest {
 
               Entry<Key,Value> entry = iter.next();
               if (!entry.getValue().toString().equals(val2) || !entry.getKey().equals(new Key(new Text(row2)))) {
-                log.error("nm.iter(" + row2 + ") failed row = " + row + " count = " + count + " row2 = " + row + " val2 = " + val2);
+                log.error("nm.iter({}) failed row = {} count = {} row2 = {} val2 = {}", row2, row, count, row, val2);
               }
 
               count++;
@@ -122,7 +144,14 @@ public class NativeMapStressTest {
             scanned += count;
           }
 
-          opTimer.stop("Finished " + getsPerThread + " random iterations (scanned = " + scanned + ") in %DURATION%");
+          if (timer != null) {
+
+            // stop and log created elapsed time
+            timer.stop();
+            log.info("tid={} oid={} Finished {}  random iterations (scanned = {}) in {}", Thread.currentThread().getId(), nextOpid.getAndIncrement(),
+                getsPerThread, scanned, String.format("%.3f secs", timer.scale(TimeUnit.SECONDS)));
+
+          }
 
           nm.delete();
         }
@@ -138,7 +167,7 @@ public class NativeMapStressTest {
       try {
         thread.join();
       } catch (InterruptedException e) {
-        log.error("Could not join thread '" + thread.getName() + "'.", e);
+        log.error("Could not join thread '{}'.", thread.getName(), e);
         throw new RuntimeException(e);
       }
     }
@@ -200,7 +229,7 @@ public class NativeMapStressTest {
       try {
         thread.join();
       } catch (InterruptedException e) {
-        log.error("Could not join thread '" + thread.getName() + "'.", e);
+        log.error("Could not join thread '{}'.", thread.getName(), e);
         throw new RuntimeException(e);
       }
     }
@@ -261,7 +290,7 @@ public class NativeMapStressTest {
       try {
         thread.join();
       } catch (InterruptedException e) {
-        log.error("Could not join thread '" + thread.getName() + "'.", e);
+        log.error("Could not join thread '{}'.", thread.getName(), e);
         throw new RuntimeException(e);
       }
     }