You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by mb...@apache.org on 2012/04/21 03:30:03 UTC

svn commit: r1328556 - in /hbase/branches/0.89-fb/src: main/java/org/apache/hadoop/hbase/ipc/ main/java/org/apache/hadoop/hbase/monitoring/ main/java/org/apache/hadoop/hbase/regionserver/ main/java/org/apache/hadoop/hbase/util/ test/java/org/apache/had...

Author: mbautin
Date: Sat Apr 21 01:30:02 2012
New Revision: 1328556

URL: http://svn.apache.org/viewvc?rev=1328556&view=rev
Log:
[HBASE-5045] Annotation for Custom Param formatting and next() RPC call info

Summary: A method for associating pretty print classes with method calls.
These allow you to get information about a method call given the params it was
called with and what instance it was called on.

The first use case is for getting info about a next() RPC call.

Test Plan: Run a script that stresses a regionserver with scan and next() scans, and
check that the information is show in the JSON view of the TaskMonitor

Reviewers: kannan

Differential Revision: https://phabricator.fb.com/D441165

Task ID: 850300

Added:
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ParamFormat.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ParamFormatHelper.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ParamFormatter.java
    hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestParamFormatter.java
Modified:
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRPC.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandler.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandlerImpl.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTask.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
    hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRPC.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRPC.java?rev=1328556&r1=1328555&r2=1328556&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRPC.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HBaseRPC.java Sat Apr 21 01:30:02 2012
@@ -33,11 +33,11 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.util.ParamFormatHelper;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.ipc.VersionedProtocol;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.StringUtils;
 
 import org.codehaus.jackson.map.ObjectMapper;
 
@@ -45,10 +45,12 @@ import javax.net.SocketFactory;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+
 import java.lang.reflect.Array;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
+
 import java.lang.reflect.Proxy;
 import java.net.ConnectException;
 import java.net.InetSocketAddress;
@@ -428,7 +430,7 @@ public class HBaseRPC {
   throws IOException {
     VersionedProtocol proxy =
         (VersionedProtocol) Proxy.newProxyInstance(
-            protocol.getClassLoader(), new Class[] { protocol },
+            protocol.getClassLoader(), new Class[]{protocol},
             new Invoker(addr, ticket, conf, factory, rpcTimeout));
     long serverVersion = proxy.getProtocolVersion(protocol.getName(),
                                                   clientVersion);
@@ -553,6 +555,7 @@ public class HBaseRPC {
     private Object instance;
     private Class<?> implementation;
     private boolean verbose;
+    ParamFormatHelper paramFormatHelper;
 
     private static final String WARN_RESPONSE_TIME =
       "hbase.ipc.warn.response.time";
@@ -598,7 +601,8 @@ public class HBaseRPC {
      */
     public Server(Object instance, Configuration conf, String bindAddress,  int port,
                   int numHandlers, boolean verbose) throws IOException {
-      super(bindAddress, port, Invocation.class, numHandlers, conf, classNameBase(instance.getClass().getName()));
+      super(bindAddress, port, Invocation.class, numHandlers, conf,
+          classNameBase(instance.getClass().getName()));
       this.instance = instance;
       this.implementation = instance.getClass();
       this.verbose = verbose;
@@ -606,6 +610,27 @@ public class HBaseRPC {
           DEFAULT_WARN_RESPONSE_TIME);
       this.warnResponseSize = conf.getInt(WARN_RESPONSE_SIZE,
           DEFAULT_WARN_RESPONSE_SIZE);
+
+      // Create a param formatter for TaskMonitor to use to pretty print
+      //  the arguments passed over RPC. See ScanParamsFormatter for an example
+      paramFormatHelper = new ParamFormatHelper(instance);
+    }
+
+    /**
+     * Gets info about a specific RPC call given an specific method and
+     * arguments it takes. Relies on the method in the RPC handler having an
+     * ParamFormat annotation on it. If there is no pretty printer, this returns
+     * null
+     * @param method the method of this.instance to get the info on
+     * @param params the params that would be passed to this function
+     * @return A map from String to object of information on this RPC call
+     */
+    @Override
+    public Map<String, Object> getParamFormatMap(Method method, Object[] params) {
+      if (paramFormatHelper != null) {
+        return paramFormatHelper.getMap(method, params);
+      }
+      return null;
     }
 
     @Override
@@ -618,17 +643,16 @@ public class HBaseRPC {
               "cause is a version mismatch between client and server.");
         }
         if (verbose) log("Call: " + call);
-        status.setRPC(call.getMethodName(), call.getParameters(), receivedTime);
+        Method method = implementation.getMethod(call.getMethodName(),
+                call.getParameterClasses());
+        status.setRPC(call.getMethodName(), call.getParameters(), receivedTime, method);
         status.setRPCPacket(param);
         status.resume("Servicing call");
-        Method method =
-          implementation.getMethod(call.getMethodName(),
-                                   call.getParameterClasses());
 
         long startTime = System.currentTimeMillis();
         Object value = method.invoke(instance, call.getParameters());
         int processingTime = (int) (System.currentTimeMillis() - startTime);
-        int qTime = (int) (startTime-receivedTime);
+        int qTime = (int) (startTime - receivedTime);
         if (LOG.isDebugEnabled()) {
           LOG.debug("Served: " + call.getMethodName() +
             " queueTime= " + qTime +
@@ -637,7 +661,7 @@ public class HBaseRPC {
         rpcMetrics.rpcQueueTime.inc(qTime);
         rpcMetrics.rpcProcessingTime.inc(processingTime);
         rpcMetrics.inc(call.getMethodName(), processingTime);
-        if (verbose) log("Return: "+value);
+        if (verbose) log("Return: " + value);
 
         HbaseObjectWritable retVal =
           new HbaseObjectWritable(method.getReturnType(), value);

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java?rev=1328556&r1=1328555&r2=1328556&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java Sat Apr 21 01:30:02 2012
@@ -26,6 +26,7 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.lang.reflect.Method;
 import java.net.BindException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
@@ -50,6 +51,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.*;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 
@@ -141,6 +143,18 @@ public abstract class HBaseServer {
     return (addr == null) ? null : addr.getHostAddress();
   }
 
+  /**
+   * Stub method for getting information about a RPC call. In the future, we
+   *  could use this for real pretty printing of non-rpc calls.
+   * @see HBaseRPC#Server#getParamFormatMap(java.lang.reflect.Method, Object[])
+   * @param method Ignored for now
+   * @param params Ignored for now
+   * @return null
+   */
+  public Map<String, Object> getParamFormatMap(Method method, Object[] params) {
+    return null;
+  }
+
   protected String bindAddress;
   protected int port;                             // port we listen on
   private int handlerCount;                       // number of handler threads
@@ -973,6 +987,7 @@ public abstract class HBaseServer {
     public void run() {
       LOG.info(getName() + ": starting");
       status.setStatus("starting");
+      status.setProcessingServer(HBaseServer.this);
       SERVER.set(HBaseServer.this);
       while (running) {
         try {

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandler.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandler.java?rev=1328556&r1=1328555&r2=1328556&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandler.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandler.java Sat Apr 21 01:30:02 2012
@@ -22,6 +22,8 @@ package org.apache.hadoop.hbase.monitori
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.Writable;
 
+import java.lang.reflect.Method;
+
 /**
  * A MonitoredTask implementation optimized for use with RPC Handlers
  * handling frequent, short duration tasks. String concatenations and object
@@ -38,7 +40,7 @@ public interface MonitoredRPCHandler ext
   public abstract boolean isOperationRunning();
 
   public abstract void setRPC(String methodName, Object [] params,
-      long queueTime);
+      long queueTime, Method realMethod);
   public abstract void setRPCPacket(Writable param);
   public abstract void setConnection(String clientAddress, int remotePort);
 }

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandlerImpl.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandlerImpl.java?rev=1328556&r1=1328555&r2=1328556&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandlerImpl.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredRPCHandlerImpl.java Sat Apr 21 01:30:02 2012
@@ -27,6 +27,7 @@ import org.apache.hadoop.io.Writable;
 import org.codehaus.jackson.map.ObjectMapper;
 
 import java.io.IOException;
+import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Map;
@@ -45,6 +46,7 @@ public class MonitoredRPCHandlerImpl ext
   private String methodName = "";
   private Object [] params = {};
   private Writable packet;
+  private Method realMethod;
 
   public MonitoredRPCHandlerImpl() {
     super();
@@ -190,7 +192,8 @@ public class MonitoredRPCHandlerImpl ext
    * @param params The parameters that will be passed to the indicated method.
    */
   public synchronized void setRPC(String methodName, Object [] params,
-      long queueTime) {
+      long queueTime, Method realMethod) {
+    this.realMethod = realMethod;
     this.methodName = methodName;
     this.params = params;
     this.rpcStartTime = System.currentTimeMillis();
@@ -243,6 +246,17 @@ public class MonitoredRPCHandlerImpl ext
         paramList.add(param.toString());
       }
     }
+
+    // If we have a custom ParamFormatter for this method, add the info
+    //  it generates to the map
+    if (processingServer != null) {
+      Map<String, Object> moreInfo =
+        processingServer.getParamFormatMap(realMethod, params);
+      if (moreInfo != null) {
+        map.putAll(moreInfo);
+      }
+    }
+
     return map;
   }
 

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTask.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTask.java?rev=1328556&r1=1328555&r2=1328556&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTask.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTask.java Sat Apr 21 01:30:02 2012
@@ -19,6 +19,8 @@
  */
 package org.apache.hadoop.hbase.monitoring;
 
+import org.apache.hadoop.hbase.ipc.HBaseServer;
+
 import java.io.IOException;
 import java.util.Map;
 
@@ -46,6 +48,7 @@ public interface MonitoredTask extends C
 
   public abstract void setStatus(String status);
   public abstract void setDescription(String description);
+  public abstract void setProcessingServer(HBaseServer server);
 
   /**
    * Explicitly mark this status as able to be cleaned up,

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java?rev=1328556&r1=1328555&r2=1328556&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java Sat Apr 21 01:30:02 2012
@@ -19,6 +19,7 @@
  */
 package org.apache.hadoop.hbase.monitoring;
 
+import org.apache.hadoop.hbase.ipc.HBaseServer;
 import org.codehaus.jackson.map.ObjectMapper;
 
 
@@ -36,6 +37,8 @@ class MonitoredTaskImpl implements Monit
 
   protected volatile State state = State.RUNNING;
 
+  HBaseServer processingServer;
+
   public MonitoredTaskImpl() {
     startTime = System.currentTimeMillis();
     statusTime = startTime;
@@ -130,6 +133,11 @@ class MonitoredTaskImpl implements Monit
   }
 
   @Override
+  public void setProcessingServer(HBaseServer server) {
+    this.processingServer = server;
+  }
+
+  @Override
   public void cleanup() {
     if (state == State.RUNNING) {
       setState(State.ABORTED);

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java?rev=1328556&r1=1328555&r2=1328556&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java Sat Apr 21 01:30:02 2012
@@ -2902,10 +2902,13 @@ public class HRegion implements HeapSize
     private int isScan;
     private boolean filterClosed = false;
     private long readPt;
+    private Scan originalScan;
 
     RegionScanner(Scan scan, List<KeyValueScanner> additionalScanners) throws IOException {
       //DebugPrint.println("HRegionScanner.<init>");
 
+      this.originalScan = scan;
+
       this.filter = scan.getFilter();
       this.batch = scan.getBatch();
       if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
@@ -3102,6 +3105,14 @@ public class HRegion implements HeapSize
     KeyValueHeap getStoreHeapForTesting() {
       return storeHeap;
     }
+
+    /**
+     * Get the original scan object that was used to create this internal one
+     * @return original scan object... used for debug output
+     */
+    public Scan getOriginalScan() {
+      return originalScan;
+    }
   }
 
   // Utility methods

Modified: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java?rev=1328556&r1=1328555&r2=1328556&view=diff
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (original)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java Sat Apr 21 01:30:02 2012
@@ -115,13 +115,7 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics.StoreMetricType;
 import org.apache.hadoop.hbase.regionserver.wal.HLog;
 import org.apache.hadoop.hbase.replication.regionserver.Replication;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.FSUtils;
-import org.apache.hadoop.hbase.util.InfoServer;
-import org.apache.hadoop.hbase.util.Pair;
-import org.apache.hadoop.hbase.util.RuntimeHaltAbortStrategy;
-import org.apache.hadoop.hbase.util.Sleeper;
-import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.hbase.util.*;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
 import org.apache.hadoop.io.MapWritable;
 import org.apache.hadoop.io.Writable;
@@ -2160,6 +2154,7 @@ public class HRegionServer implements HR
     return scannerId;
   }
 
+  @ParamFormat(clazz = ScanParamsFormatter.class)
   @Override
   public Result next(final long scannerId) throws IOException {
     Result [] res = next(scannerId, 1);
@@ -2169,6 +2164,34 @@ public class HRegionServer implements HR
     return res[0];
   }
 
+  /**
+   * Pretty param printer for next() RPC calls. (works for 1 and 2 parameter
+   * methods)
+   * @see ParamFormatter
+   */
+  public static class ScanParamsFormatter implements ParamFormatter<HRegionServer> {
+    // TODO make this configurable - same as Operation class's
+    private static final int DEFAULT_MAX_COLS = 5;
+
+    @Override
+    public Map<String, Object> getMap(Object[] params, HRegionServer regionServer) {
+      Map<String, Object> res = new HashMap<String, Object>();
+      if (params == null || params.length == 0) return null; // bad request
+      long scannerId = (Long) params[0];
+      String scannerName = String.valueOf(scannerId);
+      InternalScanner s = regionServer.scanners.get(scannerName);
+      if (s != null && s instanceof HRegion.RegionScanner) {
+        res.put("scan", ((HRegion.RegionScanner)s).getOriginalScan().toMap(DEFAULT_MAX_COLS));
+      }
+
+      if (params.length > 1) {
+        res.put("maxrows", params[1]);
+      }
+      return res;
+    }
+  }
+
+  @ParamFormat(clazz = ScanParamsFormatter.class)
   @Override
   public Result [] next(final long scannerId, int nbRows) throws IOException {
     try {

Added: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ParamFormat.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ParamFormat.java?rev=1328556&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ParamFormat.java (added)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ParamFormat.java Sat Apr 21 01:30:02 2012
@@ -0,0 +1,44 @@
+/**
+ * Copyright 2011 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.util;
+import java.lang.annotation.*;
+
+/**
+ * Annotation for assigning a pretty printing class (implementing ParamFormatter)
+ * to a method.
+ */
+@Target({ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+@Documented
+public @interface ParamFormat {
+  public enum FormatTypes {DEFAULT, DETAILED};
+  /**
+   * The class associated with this method, to pretty print it
+   */
+  Class<? extends ParamFormatter> clazz();
+
+  /**
+   * In the future, we might use formatType to be able to have
+   * multiple types of pretty printing on one method. e.g. default or detailed
+   * To support multiple types of formats on one method, me need to add an
+   * aggregate ParamFormats annotation that takes an array of this annotation
+   */
+  FormatTypes formatType() default FormatTypes.DEFAULT;
+}

Added: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ParamFormatHelper.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ParamFormatHelper.java?rev=1328556&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ParamFormatHelper.java (added)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ParamFormatHelper.java Sat Apr 21 01:30:02 2012
@@ -0,0 +1,112 @@
+/**
+ * Copyright 2011 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.util;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Helper class that uses reflection to iterate through a class and find
+ * "pretty print" annotations of methods.  It constructs the pretty print
+ * classes, and allows you to easily call getMap given the method and params
+ */
+public class ParamFormatHelper<TContext> {
+  Map<Method, ParamFormatter<TContext>> paramFormatters =
+      new HashMap<Method, ParamFormatter<TContext>>();
+
+  public static final Log LOG = LogFactory.getLog(ParamFormatHelper.class);
+
+  TContext context;
+
+  /**
+   * General constructor which allows you to specify the context, clazz and
+   * formatType individually
+   * @param context the context to use when calling ParamFormatter instances
+   * @param clazz the Class to reflect over to get the ParamFormat annotations
+   * @param formatType which formatTypes to use (ignores others)
+   */
+  public ParamFormatHelper(TContext context, Class<?> clazz,
+                           ParamFormat.FormatTypes formatType) {
+    this.context = context;
+
+    for (Method method : clazz.getMethods()) {
+      for (Annotation ann : method.getAnnotations()) {
+
+        if (!(ann instanceof ParamFormat)) continue;
+        ParamFormat paramFormat = (ParamFormat) ann;
+        if (paramFormat.formatType() != formatType) continue;
+
+        try {
+          // getMethod throws if it wasn't derived from ParamFormatter<TContext>
+          paramFormat.clazz().getMethod("getMap", Object[].class, clazz);
+
+          // The generic constraint in the annotation checks this...
+          @SuppressWarnings("unchecked")
+          ParamFormatter<TContext> newFormatter =
+              (ParamFormatter<TContext>) paramFormat.clazz().newInstance();
+
+          paramFormatters.put(method, newFormatter);
+        }
+        catch(InstantiationException e) {
+          String msg = "Can not create " + ((ParamFormat) ann).clazz().getName() +
+              " make sure it's public (and static if nested)";
+          throw new RuntimeException(msg, e);
+        }
+        catch(NoSuchMethodException e) {
+          // This exception is thrown if the class clazz was derived
+          //  from a different TContext
+        }
+        catch (IllegalAccessException e) {
+          String msg = "Can not create " + ((ParamFormat) ann).clazz().getName() +
+              " make sure it's public (and static if nested)";
+          throw new RuntimeException(msg, e);
+        }
+      }
+    }
+
+  }
+
+  /**
+   * Default constructor that uses context as the context and class, and
+   * DEFAULT formatType
+   * @param context the context to use for calling ParamFormatter instances
+   *                and for getting the ParamFormat annotations from
+   */
+  public ParamFormatHelper(TContext context) {
+    this(context, context.getClass(), ParamFormat.FormatTypes.DEFAULT);
+  }
+
+  /**
+   * Try to get info about a method given it's params
+   * @param method which method was called
+   * @param params the params which the method was called with
+   * @return map of information about the call
+   */
+  public Map<String, Object> getMap(Method method, Object[] params) {
+    ParamFormatter<TContext> formatter = paramFormatters.get(method);
+    if (formatter == null) return null;
+    return formatter.getMap(params, context);
+  }
+}

Added: hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ParamFormatter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ParamFormatter.java?rev=1328556&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ParamFormatter.java (added)
+++ hbase/branches/0.89-fb/src/main/java/org/apache/hadoop/hbase/util/ParamFormatter.java Sat Apr 21 01:30:02 2012
@@ -0,0 +1,39 @@
+/**
+ * Copyright 2011 The Apache Software Foundation
+ *
+ * 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.hadoop.hbase.util;
+import java.util.Map;
+
+/**
+ * Interface for pretty printing information about a method call.
+ * Each implementing class is (usually) specific for one method,
+ * and is assigned with the ParamFormat annotation.
+ *
+ * @see org.apache.hadoop.hbase.regionserver.HRegionServer#ScanParamsFormatter
+ */
+public interface ParamFormatter<TContext> {
+  /**
+   * Returns information about a method call given params and context.
+   *
+   * @param params The params that are/were passed to the method
+   * @param context Usually the instance the method is called on.
+   * @return A map of information about the method call
+   */
+  Map<String, Object> getMap(Object[] params, TContext context);
+}

Added: hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestParamFormatter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestParamFormatter.java?rev=1328556&view=auto
==============================================================================
--- hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestParamFormatter.java (added)
+++ hbase/branches/0.89-fb/src/test/java/org/apache/hadoop/hbase/util/TestParamFormatter.java Sat Apr 21 01:30:02 2012
@@ -0,0 +1,146 @@
+package org.apache.hadoop.hbase.util;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.junit.Test;
+
+import java.lang.reflect.Method;
+import java.util.HashMap;
+import java.util.Map;
+
+import static junit.framework.Assert.*;
+
+/**
+ * Test the ParamFormatter framework
+ */
+public class TestParamFormatter {
+  private final static Log LOG = LogFactory.getLog(TestParamFormatter.class);
+
+
+  @Test
+  public void test1() throws Exception {
+    MyLocalServer server = new MyLocalServer();
+
+    ParamFormatHelper<MyLocalServer> simple =
+        new ParamFormatHelper<MyLocalServer>(server);
+
+    ParamFormatHelper<MyLocalServer> detailed =
+        new ParamFormatHelper<MyLocalServer>(server, server.getClass(),
+            ParamFormat.FormatTypes.DETAILED);
+
+
+
+    Object[] params1 = server.testFunction1(1, "one");
+    Object[] params2 = server.testFunction2(2, new String[]{"two"});
+
+    Method func1 =
+        server.getClass().getMethod("testFunction1", int.class, String.class);
+
+    Method func2 =
+        server.getClass().getMethod("testFunction2", long.class, String[].class);
+
+    // we don't have pretty printing for func1 (detailed)
+    assertNull(detailed.getMap(func1, params1));
+
+    // no info for func2 on simple formatType
+    assertNull(simple.getMap(func2, params2));
+
+    Map<String, Object> simpleRes = simple.getMap(func1, params1);
+    assertEquals(simpleRes.get("p1"), params1[0]);
+    assertEquals(simpleRes.get("p2"), params1[1]);
+
+    Map<String, Object> detailedRes = detailed.getMap(func2, params2);
+    assertEquals(detailedRes.get("p1"), params2[0]);
+    assertEquals(detailedRes.get("p2"), params2[1]);
+  }
+
+
+  @Test
+  public void test2() {
+    MyFailureCaseServer server = new MyFailureCaseServer();
+    try {
+      ParamFormatHelper<MyFailureCaseServer> simple =
+          new ParamFormatHelper<MyFailureCaseServer>(server);
+    }
+    catch(RuntimeException e) {
+      return;
+    }
+    fail("Should not succeed when the Pretty printer is private");
+  }
+
+  @Test
+  public void test3() {
+    try {
+      MyFailureCaseServer2 server = new MyFailureCaseServer2();
+      ParamFormatHelper<MyFailureCaseServer2> simple =
+          new ParamFormatHelper<MyFailureCaseServer2>(server);
+    }
+    catch(RuntimeException e) {
+      return;
+    }
+    fail("Should not succeed when the Pretty printer is not static");
+  }
+
+}
+
+/**
+ * Simple class (the context) for testing pretty printing of function args
+ */
+class MyLocalServer {
+  @ParamFormat(clazz = MyTwoParamFormatter.class)
+  public Object[] testFunction1(int a, String b) {
+    return new Object[] {a, b};
+  }
+
+  @ParamFormat(clazz = MyTwoParamFormatter.class, formatType = ParamFormat.FormatTypes.DETAILED)
+  public Object[] testFunction2(long param1, String[] param2) {
+    return new Object[] {param1, param2};
+  }
+
+  /**
+   * A Simple pretty printer for methods with two args. Simply returns a
+   * map with p1 -> first param and p2 -> second param
+   */
+  public static class MyTwoParamFormatter implements ParamFormatter<MyLocalServer>
+  {
+    @Override
+    public Map<String, Object> getMap(Object[] params, MyLocalServer myLocalServer) {
+      Map<String, Object> res = new HashMap<String, Object>();
+      res.put("p1", params[0]);
+      res.put("p2", params[1]);
+      return res;
+    }
+  }
+}
+
+/**
+ * Class used to check that we fail on instantiation of the pretty print class
+ */
+class MyFailureCaseServer {
+  @ParamFormat(clazz = MyPrivateParamFormatter.class)
+  public void testFunc(int a, String b) {}
+
+  // should fail because it is private
+  private static class MyPrivateParamFormatter implements ParamFormatter<MyFailureCaseServer>
+  {
+    @Override
+    public Map<String, Object> getMap(Object[] params, MyFailureCaseServer myLocalServer) {
+      return null; }
+  }
+}
+
+/**
+ * Class used to check that we fail on instantiation of the pretty print class
+ */
+class MyFailureCaseServer2 {
+  @ParamFormat(clazz = MyPrivateParamFormatter.class)
+  public void testFunc(int a, String b) {}
+
+  // should fail because it is not static
+  public class MyPrivateParamFormatter implements ParamFormatter<MyFailureCaseServer2>
+  {
+    @Override
+    public Map<String, Object> getMap(Object[] params, MyFailureCaseServer2 myLocalServer) {
+      return null; }
+  }
+}