Repository: hbase
Updated Branches:
  refs/heads/master 58888e091 -> 97ad33c69


HBASE-15609 Remove PB references from Result, DoubleColumnInterpreter and
any such public facing class for 2.0 (Ram)


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

Branch: refs/heads/master
Commit: 97ad33c6918dc39811fc86d82b38e43e60361bf5
Parents: 58888e0
Author: Ramkrishna <ramkrishna.s.vasude...@intel.com>
Authored: Mon May 9 14:56:00 2016 +0530
Committer: Ramkrishna <ramkrishna.s.vasude...@intel.com>
Committed: Mon May 9 14:56:00 2016 +0530

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hbase/ClusterId.java |   4 +-
 .../org/apache/hadoop/hbase/ClusterStatus.java  | 113 ----------------
 .../hadoop/hbase/client/AsyncProcess.java       |   6 +-
 .../hbase/client/ClusterStatusListener.java     |   3 +-
 .../apache/hadoop/hbase/client/HBaseAdmin.java  |   5 +-
 .../hadoop/hbase/client/MetricsConnection.java  |   7 +-
 .../hadoop/hbase/client/RegionLoadStats.java    |  50 ++++++++
 .../org/apache/hadoop/hbase/client/Result.java  |  20 +--
 .../hadoop/hbase/client/ResultStatsUtil.java    |   5 +-
 .../hbase/client/ServerStatisticTracker.java    |   3 +-
 .../hadoop/hbase/client/StatisticTrackable.java |   4 +-
 .../hbase/client/backoff/ServerStatistics.java  |   6 +-
 .../BigDecimalColumnInterpreter.java            |   5 +-
 .../coprocessor/DoubleColumnInterpreter.java    |   3 +-
 .../coprocessor/LongColumnInterpreter.java      |   5 +-
 .../hbase/coprocessor/ColumnInterpreter.java    |   5 +-
 .../hadoop/hbase/protobuf/ProtobufUtil.java     | 128 +++++++++++++++++++
 .../client/TestClientExponentialBackoff.java    |   5 +-
 .../org/apache/hadoop/hbase/ProcedureInfo.java  |  76 +----------
 .../org/apache/hadoop/hbase/ProcedureState.java |  30 +++++
 .../org/apache/hadoop/hbase/ProcedureUtil.java  | 103 +++++++++++++++
 .../hadoop/hbase/procedure2/Procedure.java      |  21 ++-
 .../store/wal/ProcedureWALFormatReader.java     |   3 +-
 .../procedure2/ProcedureTestingUtility.java     |  12 +-
 .../hadoop/hbase/master/MasterRpcServices.java  |   7 +-
 .../master/procedure/ProcedureSyncWait.java     |   2 +-
 .../apache/hadoop/hbase/client/TestResult.java  |   3 +-
 .../master/procedure/TestProcedureAdmin.java    |   2 +-
 28 files changed, 385 insertions(+), 251 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterId.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterId.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterId.java
index 6a3b14f..c127627 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterId.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterId.java
@@ -81,7 +81,7 @@ public class ClusterId {
   /**
    * @return A pb instance to represent this instance.
    */
-  ClusterIdProtos.ClusterId convert() {
+  public ClusterIdProtos.ClusterId convert() {
     ClusterIdProtos.ClusterId.Builder builder = 
ClusterIdProtos.ClusterId.newBuilder();
     return builder.setClusterId(this.id).build();
   }
@@ -90,7 +90,7 @@ public class ClusterId {
    * @param cid
    * @return A {@link ClusterId} made from the passed in <code>cid</code>
    */
-  static ClusterId convert(final ClusterIdProtos.ClusterId cid) {
+  public static ClusterId convert(final ClusterIdProtos.ClusterId cid) {
     return new ClusterId(cid.getClusterId());
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
index 4b73dda..bc97a95 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
@@ -346,117 +346,4 @@ public class ClusterStatus extends VersionedWritable {
     }
     return sb.toString();
   }
-
-  /**
-    * Convert a ClusterStatus to a protobuf ClusterStatus
-    *
-    * @return the protobuf ClusterStatus
-    */
-  public ClusterStatusProtos.ClusterStatus convert() {
-    ClusterStatusProtos.ClusterStatus.Builder builder =
-        ClusterStatusProtos.ClusterStatus.newBuilder();
-    
builder.setHbaseVersion(HBaseVersionFileContent.newBuilder().setVersion(getHBaseVersion()));
-
-    if (liveServers != null){
-      for (Map.Entry<ServerName, ServerLoad> entry : liveServers.entrySet()) {
-        LiveServerInfo.Builder lsi =
-          
LiveServerInfo.newBuilder().setServer(ProtobufUtil.toServerName(entry.getKey()));
-        lsi.setServerLoad(entry.getValue().obtainServerLoadPB());
-        builder.addLiveServers(lsi.build());
-      }
-    }
-
-    if (deadServers != null){
-      for (ServerName deadServer : deadServers) {
-        builder.addDeadServers(ProtobufUtil.toServerName(deadServer));
-      }
-    }
-
-    if (intransition != null) {
-      for (Map.Entry<String, RegionState> rit : 
getRegionsInTransition().entrySet()) {
-        ClusterStatusProtos.RegionState rs = rit.getValue().convert();
-        RegionSpecifier.Builder spec =
-            
RegionSpecifier.newBuilder().setType(RegionSpecifierType.REGION_NAME);
-        spec.setValue(ByteStringer.wrap(Bytes.toBytes(rit.getKey())));
-
-        RegionInTransition pbRIT =
-            
RegionInTransition.newBuilder().setSpec(spec.build()).setRegionState(rs).build();
-        builder.addRegionsInTransition(pbRIT);
-      }
-    }
-
-    if (clusterId != null) {
-      builder.setClusterId(new ClusterId(clusterId).convert());
-    }
-
-    if (masterCoprocessors != null) {
-      for (String coprocessor : masterCoprocessors) {
-        
builder.addMasterCoprocessors(HBaseProtos.Coprocessor.newBuilder().setName(coprocessor));
-      }
-    }
-
-    if (master != null){
-      builder.setMaster(ProtobufUtil.toServerName(getMaster()));
-    }
-
-    if (backupMasters != null) {
-      for (ServerName backup : backupMasters) {
-        builder.addBackupMasters(ProtobufUtil.toServerName(backup));
-      }
-    }
-
-    if (balancerOn != null){
-      builder.setBalancerOn(balancerOn);
-    }
-
-    return builder.build();
-  }
-
-  /**
-   * Convert a protobuf ClusterStatus to a ClusterStatus
-   *
-   * @param proto the protobuf ClusterStatus
-   * @return the converted ClusterStatus
-   */
-  public static ClusterStatus convert(ClusterStatusProtos.ClusterStatus proto) 
{
-
-    Map<ServerName, ServerLoad> servers = null;
-    servers = new HashMap<ServerName, 
ServerLoad>(proto.getLiveServersList().size());
-    for (LiveServerInfo lsi : proto.getLiveServersList()) {
-      servers.put(ProtobufUtil.toServerName(
-          lsi.getServer()), new ServerLoad(lsi.getServerLoad()));
-    }
-
-    Collection<ServerName> deadServers = null;
-    deadServers = new ArrayList<ServerName>(proto.getDeadServersList().size());
-    for (HBaseProtos.ServerName sn : proto.getDeadServersList()) {
-      deadServers.add(ProtobufUtil.toServerName(sn));
-    }
-
-    Collection<ServerName> backupMasters = null;
-    backupMasters = new 
ArrayList<ServerName>(proto.getBackupMastersList().size());
-    for (HBaseProtos.ServerName sn : proto.getBackupMastersList()) {
-      backupMasters.add(ProtobufUtil.toServerName(sn));
-    }
-
-    Map<String, RegionState> rit = null;
-    rit = new HashMap<String, 
RegionState>(proto.getRegionsInTransitionList().size());
-    for (RegionInTransition region : proto.getRegionsInTransitionList()) {
-      String key = new String(region.getSpec().getValue().toByteArray());
-      RegionState value = RegionState.convert(region.getRegionState());
-      rit.put(key, value);
-    }
-
-    String[] masterCoprocessors = null;
-    final int numMasterCoprocessors = proto.getMasterCoprocessorsCount();
-    masterCoprocessors = new String[numMasterCoprocessors];
-    for (int i = 0; i < numMasterCoprocessors; i++) {
-      masterCoprocessors[i] = proto.getMasterCoprocessors(i).getName();
-    }
-
-    return new ClusterStatus(proto.getHbaseVersion().getVersion(),
-      ClusterId.convert(proto.getClusterId()).toString(),servers,deadServers,
-      
ProtobufUtil.toServerName(proto.getMaster()),backupMasters,rit,masterCoprocessors,
-      proto.getBalancerOn());
-  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
index b2c758d..6de8c82 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
@@ -55,6 +55,7 @@ import 
org.apache.hadoop.hbase.client.backoff.ServerStatistics;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -1718,10 +1719,11 @@ class AsyncProcess {
     for (Map.Entry<byte[], MultiResponse.RegionResult> regionStats : 
results.entrySet()) {
       byte[] regionName = regionStats.getKey();
       ClientProtos.RegionLoadStats stat = regionStats.getValue().getStat();
+      RegionLoadStats regionLoadstats = 
ProtobufUtil.createRegionLoadStats(stat);
       
ResultStatsUtil.updateStats(AsyncProcess.this.connection.getStatisticsTracker(),
 server,
-          regionName, stat);
+          regionName, regionLoadstats);
       
ResultStatsUtil.updateStats(AsyncProcess.this.connection.getConnectionMetrics(),
-          server, regionName, stat);
+          server, regionName, regionLoadstats);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
index 5756232..9efb33d 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterStatusListener.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.ExceptionUtil;
@@ -257,7 +258,7 @@ class ClusterStatusListener implements Closeable {
         ByteBufInputStream bis = new ByteBufInputStream(dp.content());
         try {
           ClusterStatusProtos.ClusterStatus csp = 
ClusterStatusProtos.ClusterStatus.parseFrom(bis);
-          ClusterStatus ncs = ClusterStatus.convert(csp);
+          ClusterStatus ncs = ProtobufUtil.convert(csp);
           receive(ncs);
         } finally {
           bis.close();

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 4b0609d..a2ef34b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.NotServingRegionException;
 import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.ProcedureUtil;
 import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableExistsException;
@@ -1847,7 +1848,7 @@ public class HBaseAdmin implements Admin {
         PayloadCarryingRpcController controller = 
rpcControllerFactory.newController();
         controller.setCallTimeout(callTimeout);
         GetClusterStatusRequest req = 
RequestConverter.buildGetClusterStatusRequest();
-        return ClusterStatus.convert(master.getClusterStatus(controller, 
req).getClusterStatus());
+        return ProtobufUtil.convert(master.getClusterStatus(controller, 
req).getClusterStatus());
       }
     });
   }
@@ -2013,7 +2014,7 @@ public class HBaseAdmin implements Admin {
               controller, 
ListProceduresRequest.newBuilder().build()).getProcedureList();
             ProcedureInfo[] procInfoList = new ProcedureInfo[procList.size()];
             for (int i = 0; i < procList.size(); i++) {
-              procInfoList[i] = ProcedureInfo.convert(procList.get(i));
+              procInfoList[i] = ProcedureUtil.convert(procList.get(i));
             }
             return procInfoList;
           }

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
index 2839fb0..a5dc7fb 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MetricsConnection.java
@@ -28,7 +28,6 @@ import com.codahale.metrics.JmxReporter;
 import com.codahale.metrics.RatioGauge;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
 import 
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
@@ -156,7 +155,7 @@ public class MetricsConnection implements 
StatisticTrackable {
           HEAP_BASE + this.name));
     }
 
-    public void update(ClientProtos.RegionLoadStats regionStatistics) {
+    public void update(RegionLoadStats regionStatistics) {
       this.memstoreLoadHist.update(regionStatistics.getMemstoreLoad());
       this.heapOccupancyHist.update(regionStatistics.getHeapOccupancy());
     }
@@ -200,7 +199,7 @@ public class MetricsConnection implements 
StatisticTrackable {
       return;
     }
     Result result = (Result) r;
-    ClientProtos.RegionLoadStats stats = result.getStats();
+    RegionLoadStats stats = result.getStats();
     if (stats == null) {
       return;
     }
@@ -209,7 +208,7 @@ public class MetricsConnection implements 
StatisticTrackable {
 
   @Override
   public void updateRegionStats(ServerName serverName, byte[] regionName,
-    ClientProtos.RegionLoadStats stats) {
+    RegionLoadStats stats) {
     String name = serverName.getServerName() + "," + 
Bytes.toStringBinary(regionName);
     ConcurrentMap<byte[], RegionStats> rsStats = null;
     if (serverStats.containsKey(serverName)) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLoadStats.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLoadStats.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLoadStats.java
new file mode 100644
index 0000000..443026f
--- /dev/null
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionLoadStats.java
@@ -0,0 +1,50 @@
+/**
+ * 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.client;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+/**
+ * POJO representing region server load
+ */
+public class RegionLoadStats {
+  int memstoreLoad;
+  int heapOccupancy;
+  int compactionPressure;
+
+  public RegionLoadStats(int memstoreLoad, int heapOccupancy, int 
compactionPressure) {
+    this.memstoreLoad = memstoreLoad;
+    this.heapOccupancy = heapOccupancy;
+    this.compactionPressure = compactionPressure;
+  }
+
+  public int getMemstoreLoad() {
+    return this.memstoreLoad;
+  }
+
+  public int getHeapOccupancy() {
+    return this.heapOccupancy;
+  }
+
+  public int getCompactionPressure() {
+    return this.compactionPressure;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
index d2a49c2..c7393f6 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Result.java
@@ -111,7 +111,7 @@ public class Result implements CellScannable, CellScanner {
    * Index for where we are when Result is acting as a {@link CellScanner}.
    */
   private int cellScannerIndex = INITIAL_CELLSCANNER_INDEX;
-  private ClientProtos.RegionLoadStats stats;
+  private RegionLoadStats stats;
 
   private final boolean readonly;
 
@@ -909,25 +909,11 @@ public class Result implements CellScannable, CellScanner 
{
   }
 
   /**
-   * Add load information about the region to the information about the result
-   * @param loadStats statistics about the current region from which this was 
returned
-   * @deprecated use {@link #setStatistics(ClientProtos.RegionLoadStats)} 
instead
-   * @throws UnsupportedOperationException if invoked on instance of 
EMPTY_RESULT
-   * (which is supposed to be immutable).
-   */
-  @InterfaceAudience.Private
-  @Deprecated
-  public void addResults(ClientProtos.RegionLoadStats loadStats) {
-    checkReadonly();
-    this.stats = loadStats;
-  }
-
-  /**
    * Set load information about the region to the information about the result
    * @param loadStats statistics about the current region from which this was 
returned
    */
   @InterfaceAudience.Private
-  public void setStatistics(ClientProtos.RegionLoadStats loadStats) {
+  public void setStatistics(RegionLoadStats loadStats) {
     this.stats = loadStats;
   }
 
@@ -935,7 +921,7 @@ public class Result implements CellScannable, CellScanner {
    * @return the associated statistics about the region from which this was 
returned. Can be
    * <tt>null</tt> if stats are disabled.
    */
-  public ClientProtos.RegionLoadStats getStats() {
+  public RegionLoadStats getStats() {
     return stats;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultStatsUtil.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultStatsUtil.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultStatsUtil.java
index 6537d79..b5bf6c1 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultStatsUtil.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultStatsUtil.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.client;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 
 /**
  * A {@link Result} with some statistics about the server/region status
@@ -50,7 +49,7 @@ public final class ResultStatsUtil {
     }
     Result result = (Result) r;
     // early exit if there are no stats to collect
-    ClientProtos.RegionLoadStats stats = result.getStats();
+    RegionLoadStats stats = result.getStats();
     if(stats == null){
       return r;
     }
@@ -60,7 +59,7 @@ public final class ResultStatsUtil {
   }
 
   public static void updateStats(StatisticTrackable tracker, ServerName 
server, byte[] regionName,
-    ClientProtos.RegionLoadStats stats) {
+    RegionLoadStats stats) {
     if (regionName != null && stats != null && tracker != null) {
       tracker.updateRegionStats(server, regionName, stats);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerStatisticTracker.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerStatisticTracker.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerStatisticTracker.java
index b8e7923..cb21e8b 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerStatisticTracker.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ServerStatisticTracker.java
@@ -23,7 +23,6 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 
 import java.util.concurrent.ConcurrentHashMap;
 
@@ -37,7 +36,7 @@ public class ServerStatisticTracker implements 
StatisticTrackable {
       new ConcurrentHashMap<ServerName, ServerStatistics>();
 
   @Override
-  public void updateRegionStats(ServerName server, byte[] region, 
ClientProtos.RegionLoadStats
+  public void updateRegionStats(ServerName server, byte[] region, 
RegionLoadStats
       currentStats) {
     ServerStatistics stat = stats.get(server);
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/StatisticTrackable.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/StatisticTrackable.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/StatisticTrackable.java
index 7bb49e7..69f06ff 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/StatisticTrackable.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/StatisticTrackable.java
@@ -18,7 +18,6 @@ package org.apache.hadoop.hbase.client;
 
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 
 /**
  * Parent interface for an object to get updates about per-region statistics.
@@ -28,6 +27,5 @@ public interface StatisticTrackable {
   /**
    *  Update stats per region.
    * */
-  void updateRegionStats(ServerName server, byte[] region, 
ClientProtos.RegionLoadStats
-    stats);
+  void updateRegionStats(ServerName server, byte[] region, RegionLoadStats 
stats);
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ServerStatistics.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ServerStatistics.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ServerStatistics.java
index 2072573..e33e2bc 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ServerStatistics.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/backoff/ServerStatistics.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.hbase.client.backoff;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.client.RegionLoadStats;
 import org.apache.hadoop.hbase.util.Bytes;
 
 import java.util.Map;
@@ -39,7 +39,7 @@ public class ServerStatistics {
    * @param region
    * @param currentStats
    */
-  public void update(byte[] region, ClientProtos.RegionLoadStats currentStats) 
{
+  public void update(byte[] region, RegionLoadStats currentStats) {
     RegionStatistics regionStat = this.stats.get(region);
     if(regionStat == null){
       regionStat = new RegionStatistics();
@@ -59,7 +59,7 @@ public class ServerStatistics {
     private int heapOccupancy = 0;
     private int compactionPressure = 0;
 
-    public void update(ClientProtos.RegionLoadStats currentStats) {
+    public void update(RegionLoadStats currentStats) {
       this.memstoreLoad = currentStats.getMemstoreLoad();
       this.heapOccupancy = currentStats.getHeapOccupancy();
       this.compactionPressure = currentStats.getCompactionPressure();

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java
index 7d0715b..5d1cc91 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/BigDecimalColumnInterpreter.java
@@ -24,7 +24,9 @@ import java.math.RoundingMode;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BigDecimalMsg;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg;
@@ -36,7 +38,8 @@ import org.apache.hadoop.hbase.util.Bytes;
  * is required at the RegionServer also.
  *
  */
-@InterfaceAudience.Private
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
 public class BigDecimalColumnInterpreter extends ColumnInterpreter<BigDecimal, 
BigDecimal,
   EmptyMsg, BigDecimalMsg, BigDecimalMsg> {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/DoubleColumnInterpreter.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/DoubleColumnInterpreter.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/DoubleColumnInterpreter.java
index 5b8571f..c33bb00 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/DoubleColumnInterpreter.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/DoubleColumnInterpreter.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
@@ -37,7 +38,7 @@ import org.apache.hadoop.hbase.util.Bytes;
  * TestDoubleColumnInterpreter methods for its sample usage.
  * Its methods handle null arguments gracefully. 
  */
-@InterfaceAudience.Public
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
 @InterfaceStability.Evolving
 public class DoubleColumnInterpreter extends ColumnInterpreter<Double, Double, 
       EmptyMsg, DoubleMsg, DoubleMsg>{

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java
index 586575a..f738b10 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/coprocessor/LongColumnInterpreter.java
@@ -22,7 +22,9 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LongMsg;
@@ -36,7 +38,8 @@ import org.apache.hadoop.hbase.util.Bytes;
  * TestAggregateProtocol methods for its sample usage.
  * Its methods handle null arguments gracefully. 
  */
-@InterfaceAudience.Private
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
 public class LongColumnInterpreter extends ColumnInterpreter<Long, Long,
                  EmptyMsg, LongMsg, LongMsg> {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/ColumnInterpreter.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/ColumnInterpreter.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/ColumnInterpreter.java
index e247c08..e22bd24 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/ColumnInterpreter.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/coprocessor/ColumnInterpreter.java
@@ -22,7 +22,9 @@ package org.apache.hadoop.hbase.coprocessor;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
 
 import com.google.protobuf.Message;
 
@@ -53,7 +55,8 @@ import com.google.protobuf.Message;
  * @param Q PB message that is used to transport Cell (&lt;T&gt;) instance
  * @param R PB message that is used to transport Promoted (&lt;S&gt;) instance
  */
-@InterfaceAudience.Private
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
+@InterfaceStability.Evolving
 public abstract class ColumnInterpreter<T, S, P extends Message, 
 Q extends Message, R extends Message> {
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
index 62dfd45..9c0dd6a 100644
--- 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
@@ -44,6 +44,8 @@ import static 
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpeci
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.ClusterId;
+import org.apache.hadoop.hbase.ClusterStatus;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -52,6 +54,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.Tag;
@@ -66,6 +69,7 @@ import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.Increment;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionLoadStats;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.client.SnapshotDescription;
@@ -77,6 +81,7 @@ import org.apache.hadoop.hbase.filter.ByteArrayComparable;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.io.LimitInputStream;
 import org.apache.hadoop.hbase.io.TimeRange;
+import org.apache.hadoop.hbase.master.RegionState;
 import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
 import 
org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
@@ -113,7 +118,10 @@ import 
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Del
 import 
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
+import 
org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.LiveServerInfo;
+import 
org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionInTransition;
 import 
org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
+import 
org.apache.hadoop.hbase.protobuf.generated.FSProtos.HBaseVersionFileContent;
 import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
 import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
@@ -3492,4 +3500,124 @@ public final class ProtobufUtil {
         createSnapshotType(snapshotDesc.getType()), snapshotDesc.getOwner(),
         snapshotDesc.getCreationTime(), snapshotDesc.getVersion());
   }
+
+  /**
+   * Convert a protobuf ClusterStatus to a ClusterStatus
+   *
+   * @param proto the protobuf ClusterStatus
+   * @return the converted ClusterStatus
+   */
+  public static ClusterStatus convert(ClusterStatusProtos.ClusterStatus proto) 
{
+
+    Map<ServerName, ServerLoad> servers = null;
+    servers = new HashMap<ServerName, 
ServerLoad>(proto.getLiveServersList().size());
+    for (LiveServerInfo lsi : proto.getLiveServersList()) {
+      servers.put(ProtobufUtil.toServerName(
+          lsi.getServer()), new ServerLoad(lsi.getServerLoad()));
+    }
+
+    Collection<ServerName> deadServers = null;
+    deadServers = new ArrayList<ServerName>(proto.getDeadServersList().size());
+    for (HBaseProtos.ServerName sn : proto.getDeadServersList()) {
+      deadServers.add(ProtobufUtil.toServerName(sn));
+    }
+
+    Collection<ServerName> backupMasters = null;
+    backupMasters = new 
ArrayList<ServerName>(proto.getBackupMastersList().size());
+    for (HBaseProtos.ServerName sn : proto.getBackupMastersList()) {
+      backupMasters.add(ProtobufUtil.toServerName(sn));
+    }
+
+    Map<String, RegionState> rit = null;
+    rit = new HashMap<String, 
RegionState>(proto.getRegionsInTransitionList().size());
+    for (RegionInTransition region : proto.getRegionsInTransitionList()) {
+      String key = new String(region.getSpec().getValue().toByteArray());
+      RegionState value = RegionState.convert(region.getRegionState());
+      rit.put(key, value);
+    }
+
+    String[] masterCoprocessors = null;
+    final int numMasterCoprocessors = proto.getMasterCoprocessorsCount();
+    masterCoprocessors = new String[numMasterCoprocessors];
+    for (int i = 0; i < numMasterCoprocessors; i++) {
+      masterCoprocessors[i] = proto.getMasterCoprocessors(i).getName();
+    }
+
+    return new ClusterStatus(proto.getHbaseVersion().getVersion(),
+      ClusterId.convert(proto.getClusterId()).toString(),servers,deadServers,
+      
ProtobufUtil.toServerName(proto.getMaster()),backupMasters,rit,masterCoprocessors,
+      proto.getBalancerOn());
+  }
+
+  /**
+   * Convert a ClusterStatus to a protobuf ClusterStatus
+   *
+   * @return the protobuf ClusterStatus
+   */
+  public static ClusterStatusProtos.ClusterStatus convert(ClusterStatus 
status) {
+    ClusterStatusProtos.ClusterStatus.Builder builder =
+        ClusterStatusProtos.ClusterStatus.newBuilder();
+    builder
+        
.setHbaseVersion(HBaseVersionFileContent.newBuilder().setVersion(status.getHBaseVersion()));
+
+    if (status.getServers() != null) {
+      for (ServerName serverName : status.getServers()) {
+        LiveServerInfo.Builder lsi =
+            
LiveServerInfo.newBuilder().setServer(ProtobufUtil.toServerName(serverName));
+        status.getLoad(serverName);
+        lsi.setServerLoad(status.getLoad(serverName).obtainServerLoadPB());
+        builder.addLiveServers(lsi.build());
+      }
+    }
+
+    if (status.getDeadServerNames() != null) {
+      for (ServerName deadServer : status.getDeadServerNames()) {
+        builder.addDeadServers(ProtobufUtil.toServerName(deadServer));
+      }
+    }
+
+    if (status.getRegionsInTransition() != null) {
+      for (Map.Entry<String, RegionState> rit : 
status.getRegionsInTransition().entrySet()) {
+        ClusterStatusProtos.RegionState rs = rit.getValue().convert();
+        RegionSpecifier.Builder spec =
+            
RegionSpecifier.newBuilder().setType(RegionSpecifierType.REGION_NAME);
+        spec.setValue(ByteStringer.wrap(Bytes.toBytes(rit.getKey())));
+
+        RegionInTransition pbRIT =
+            
RegionInTransition.newBuilder().setSpec(spec.build()).setRegionState(rs).build();
+        builder.addRegionsInTransition(pbRIT);
+      }
+    }
+
+    if (status.getClusterId() != null) {
+      builder.setClusterId(new ClusterId(status.getClusterId()).convert());
+    }
+
+    if (status.getMasterCoprocessors() != null) {
+      for (String coprocessor : status.getMasterCoprocessors()) {
+        
builder.addMasterCoprocessors(HBaseProtos.Coprocessor.newBuilder().setName(coprocessor));
+      }
+    }
+
+    if (status.getMaster() != null) {
+      builder.setMaster(ProtobufUtil.toServerName(status.getMaster()));
+    }
+
+    if (status.getBackupMasters() != null) {
+      for (ServerName backup : status.getBackupMasters()) {
+        builder.addBackupMasters(ProtobufUtil.toServerName(backup));
+      }
+    }
+
+    if (status.getBalancerOn() != null) {
+      builder.setBalancerOn(status.getBalancerOn());
+    }
+
+    return builder.build();
+  }
+
+  public static RegionLoadStats 
createRegionLoadStats(ClientProtos.RegionLoadStats stats) {
+    return new RegionLoadStats(stats.getMemstoreLoad(), 
stats.getHeapOccupancy(),
+        stats.getCompactionPressure());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientExponentialBackoff.java
----------------------------------------------------------------------
diff --git 
a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientExponentialBackoff.java
 
b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientExponentialBackoff.java
index dcd4fca..838c40e 100644
--- 
a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientExponentialBackoff.java
+++ 
b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientExponentialBackoff.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.backoff.ExponentialClientBackoffPolicy;
 import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
@@ -157,7 +158,7 @@ public class TestClientExponentialBackoff {
     ClientProtos.RegionLoadStats stat = 
ClientProtos.RegionLoadStats.newBuilder()
         .setMemstoreLoad
             (load).build();
-    stats.update(regionname, stat);
+    stats.update(regionname, ProtobufUtil.createRegionLoadStats(stat));
   }
 
   private void update(ServerStatistics stats, int memstoreLoad, int 
heapOccupancy,
@@ -167,6 +168,6 @@ public class TestClientExponentialBackoff {
         .setHeapOccupancy(heapOccupancy)
         .setCompactionPressure(compactionPressure)
             .build();
-    stats.update(regionname, stat);
+    stats.update(regionname, ProtobufUtil.createRegionLoadStats(stat));
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
----------------------------------------------------------------------
diff --git 
a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
index 562ad7d..5bbff87 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
@@ -22,11 +22,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import 
org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
-import 
org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
 import org.apache.hadoop.hbase.security.User;
-import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.NonceKey;
@@ -44,7 +40,7 @@ public class ProcedureInfo implements Cloneable {
   private final ProcedureState procState;
   private final long parentId;
   private final NonceKey nonceKey;
-  private final ForeignExceptionMessage exception;
+  private final ProcedureUtil.ForeignExceptionMsg exception;
   private final long lastUpdate;
   private final long startTime;
   private final byte[] result;
@@ -59,7 +55,7 @@ public class ProcedureInfo implements Cloneable {
       final ProcedureState procState,
       final long parentId,
       final NonceKey nonceKey,
-      final ForeignExceptionMessage exception,
+      final ProcedureUtil.ForeignExceptionMsg exception,
       final long lastUpdate,
       final long startTime,
       final byte[] result) {
@@ -156,24 +152,24 @@ public class ProcedureInfo implements Cloneable {
 
   public IOException getException() {
     if (isFailed()) {
-      return ForeignExceptionUtil.toIOException(exception);
+      return 
ForeignExceptionUtil.toIOException(exception.getForeignExchangeMessage());
     }
     return null;
   }
 
   @InterfaceAudience.Private
-  public ForeignExceptionMessage getForeignExceptionMessage() {
+  public ProcedureUtil.ForeignExceptionMsg getForeignExceptionMessage() {
     return exception;
   }
 
   public String getExceptionCause() {
     assert isFailed();
-    return exception.getGenericException().getClassName();
+    return 
exception.getForeignExchangeMessage().getGenericException().getClassName();
   }
 
   public String getExceptionMessage() {
     assert isFailed();
-    return exception.getGenericException().getMessage();
+    return 
exception.getForeignExchangeMessage().getGenericException().getMessage();
   }
 
   public String getExceptionFullMessage() {
@@ -217,65 +213,6 @@ public class ProcedureInfo implements Cloneable {
   }
 
   /**
-   * @return Convert the current {@link ProcedureInfo} into a Protocol Buffers 
Procedure
-   * instance.
-   */
-  @InterfaceAudience.Private
-  public static ProcedureProtos.Procedure convertToProcedureProto(
-      final ProcedureInfo procInfo) {
-    ProcedureProtos.Procedure.Builder builder = 
ProcedureProtos.Procedure.newBuilder();
-
-    builder.setClassName(procInfo.getProcName());
-    builder.setProcId(procInfo.getProcId());
-    builder.setStartTime(procInfo.getStartTime());
-    builder.setState(procInfo.getProcState());
-    builder.setLastUpdate(procInfo.getLastUpdate());
-
-    if (procInfo.hasParentId()) {
-      builder.setParentId(procInfo.getParentId());
-    }
-
-    if (procInfo.getProcOwner() != null) {
-       builder.setOwner(procInfo.getProcOwner());
-    }
-
-    if (procInfo.isFailed()) {
-        builder.setException(procInfo.getForeignExceptionMessage());
-    }
-
-    if (procInfo.hasResultData()) {
-      builder.setResult(ByteStringer.wrap(procInfo.getResult()));
-    }
-
-    return builder.build();
-  }
-
-  /**
-   * Helper to convert the protobuf object.
-   * @return Convert the current Protocol Buffers Procedure to {@link 
ProcedureInfo}
-   * instance.
-   */
-  @InterfaceAudience.Private
-  public static ProcedureInfo convert(final ProcedureProtos.Procedure 
procProto) {
-    NonceKey nonceKey = null;
-    if (procProto.getNonce() != HConstants.NO_NONCE) {
-      nonceKey = new NonceKey(procProto.getNonceGroup(), procProto.getNonce());
-    }
-
-    return new ProcedureInfo(
-      procProto.getProcId(),
-      procProto.getClassName(),
-      procProto.getOwner(),
-      procProto.getState(),
-      procProto.hasParentId() ? procProto.getParentId() : -1,
-      nonceKey,
-      procProto.hasException() ? procProto.getException() : null,
-      procProto.getLastUpdate(),
-      procProto.getStartTime(),
-      procProto.hasResult() ? procProto.getResult().toByteArray() : null);
-  }
-
-  /**
    * Check if the user is this procedure's owner
    * @param procInfo the procedure to check
    * @param user the user
@@ -293,4 +230,5 @@ public class ProcedureInfo implements Cloneable {
     }
     return procOwner.equals(user.getShortName());
   }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureState.java
----------------------------------------------------------------------
diff --git 
a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureState.java 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureState.java
new file mode 100644
index 0000000..306d285
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureState.java
@@ -0,0 +1,30 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * POJO representing Procedure State
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public enum ProcedureState {
+  INITIALIZING, RUNNABLE, WAITING, WAITING_TIMEOUT, ROLLEDBACK, FINISHED;
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureUtil.java
----------------------------------------------------------------------
diff --git 
a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureUtil.java 
b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureUtil.java
new file mode 100644
index 0000000..2c935f3
--- /dev/null
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureUtil.java
@@ -0,0 +1,103 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
+import 
org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage;
+import org.apache.hadoop.hbase.util.ByteStringer;
+import org.apache.hadoop.hbase.util.NonceKey;
+
+@InterfaceAudience.Private
+
+/**
+ * Helper to convert to/from ProcedureProtos
+ */
+public class ProcedureUtil {
+
+  private ProcedureUtil() {
+  }
+  /**
+   * @return Convert the current {@link ProcedureInfo} into a Protocol Buffers 
Procedure
+   * instance.
+   */
+  @InterfaceAudience.Private
+  public static ProcedureProtos.Procedure convertToProcedureProto(final 
ProcedureInfo procInfo) {
+    ProcedureProtos.Procedure.Builder builder = 
ProcedureProtos.Procedure.newBuilder();
+
+    builder.setClassName(procInfo.getProcName());
+    builder.setProcId(procInfo.getProcId());
+    builder.setStartTime(procInfo.getStartTime());
+    
builder.setState(ProcedureProtos.ProcedureState.valueOf(procInfo.getProcState().name()));
+    builder.setLastUpdate(procInfo.getLastUpdate());
+
+    if (procInfo.hasParentId()) {
+      builder.setParentId(procInfo.getParentId());
+    }
+
+    if (procInfo.getProcOwner() != null) {
+      builder.setOwner(procInfo.getProcOwner());
+    }
+
+    if (procInfo.isFailed()) {
+      
builder.setException(procInfo.getForeignExceptionMessage().getForeignExchangeMessage());
+    }
+
+    if (procInfo.hasResultData()) {
+      builder.setResult(ByteStringer.wrap(procInfo.getResult()));
+    }
+
+    return builder.build();
+  }
+
+  /**
+   * Helper to convert the protobuf object.
+   * @return Convert the current Protocol Buffers Procedure to {@link 
ProcedureInfo}
+   * instance.
+   */
+  @InterfaceAudience.Private
+  public static ProcedureInfo convert(final ProcedureProtos.Procedure 
procProto) {
+    NonceKey nonceKey = null;
+    if (procProto.getNonce() != HConstants.NO_NONCE) {
+      nonceKey = new NonceKey(procProto.getNonceGroup(), procProto.getNonce());
+    }
+
+    return new ProcedureInfo(procProto.getProcId(), procProto.getClassName(), 
procProto.getOwner(),
+        convertToProcedureState(procProto.getState()),
+        procProto.hasParentId() ? procProto.getParentId() : -1, nonceKey,
+        procProto.hasException() ? new 
ForeignExceptionMsg(procProto.getException()) : null,
+        procProto.getLastUpdate(), procProto.getStartTime(),
+        procProto.hasResult() ? procProto.getResult().toByteArray() : null);
+  }
+
+  public static ProcedureState 
convertToProcedureState(ProcedureProtos.ProcedureState state) {
+    return ProcedureState.valueOf(state.name());
+  }
+
+  public static class ForeignExceptionMsg {
+    private ForeignExceptionMessage exception;
+
+    public ForeignExceptionMsg(ForeignExceptionMessage exception) {
+      this.exception = exception;
+    }
+
+    public ForeignExceptionMessage getForeignExchangeMessage() {
+      return this.exception;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
----------------------------------------------------------------------
diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
index 2cc67be..7e58420 100644
--- 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/Procedure.java
@@ -29,6 +29,7 @@ import java.util.Map;
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.ProcedureUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
@@ -688,18 +689,14 @@ public abstract class Procedure<TEnvironment> implements 
Comparable<Procedure> {
   @InterfaceAudience.Private
   public static ProcedureInfo createProcedureInfo(final Procedure proc, final 
NonceKey nonceKey) {
     RemoteProcedureException exception = proc.hasException() ? 
proc.getException() : null;
-    return new ProcedureInfo(
-      proc.getProcId(),
-      proc.toStringClass(),
-      proc.getOwner(),
-      proc.getState(),
-      proc.hasParent() ? proc.getParentProcId() : -1,
-      nonceKey,
-      exception != null ?
-          RemoteProcedureException.toProto(exception.getSource(), 
exception.getCause()) : null,
-      proc.getLastUpdate(),
-      proc.getStartTime(),
-      proc.getResult());
+    return new ProcedureInfo(proc.getProcId(), proc.toStringClass(), 
proc.getOwner(),
+        ProcedureUtil.convertToProcedureState(proc.getState()),
+        proc.hasParent() ? proc.getParentProcId() : -1, nonceKey,
+        exception != null
+            ? new ProcedureUtil.ForeignExceptionMsg(
+                RemoteProcedureException.toProto(exception.getSource(), 
exception.getCause()))
+            : null,
+        proc.getLastUpdate(), proc.getStartTime(), proc.getResult());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java
----------------------------------------------------------------------
diff --git 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java
 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java
index 312eedb..b1b201b 100644
--- 
a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java
+++ 
b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormatReader.java
@@ -24,6 +24,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.ProcedureUtil;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.procedure2.Procedure;
@@ -288,7 +289,7 @@ public class ProcedureWALFormatReader {
     }
 
     public ProcedureInfo convertToInfo() {
-      return ProcedureInfo.convert(proto);
+      return ProcedureUtil.convert(proto);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
----------------------------------------------------------------------
diff --git 
a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
 
b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
index 9edaec9..4cf48d2 100644
--- 
a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
+++ 
b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/ProcedureTestingUtility.java
@@ -175,9 +175,12 @@ public class ProcedureTestingUtility {
   }
 
   public static void assertProcNotFailed(final ProcedureInfo result) {
-    ForeignExceptionMessage exception = result.getForeignExceptionMessage();
-    String msg = exception != null ? result.getExceptionFullMessage() : "no 
exception found";
-    assertFalse(msg, result.isFailed());
+    if (result.getForeignExceptionMessage() != null) {
+      ForeignExceptionMessage exception =
+          result.getForeignExceptionMessage().getForeignExchangeMessage();
+      String msg = exception != null ? result.getExceptionFullMessage() : "no 
exception found";
+      assertFalse(msg, result.isFailed());
+    }
   }
 
   public static void assertIsAbortException(final ProcedureInfo result) {
@@ -204,7 +207,8 @@ public class ProcedureTestingUtility {
 
   public static Throwable getExceptionCause(final ProcedureInfo procInfo) {
     assert procInfo.getForeignExceptionMessage() != null;
-    return 
RemoteProcedureException.fromProto(procInfo.getForeignExceptionMessage()).getCause();
+    return RemoteProcedureException
+        
.fromProto(procInfo.getForeignExceptionMessage().getForeignExchangeMessage()).getCause();
   }
 
   public static class TestProcedure extends Procedure<Void> {

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 8a040fa..a281900 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
 import org.apache.hadoop.hbase.PleaseHoldException;
 import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.ProcedureUtil;
 import org.apache.hadoop.hbase.ServerLoad;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -707,7 +708,7 @@ public class MasterRpcServices extends RSRpcServices
     GetClusterStatusResponse.Builder response = 
GetClusterStatusResponse.newBuilder();
     try {
       master.checkInitialized();
-      response.setClusterStatus(master.getClusterStatus().convert());
+      
response.setClusterStatus(ProtobufUtil.convert(master.getClusterStatus()));
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -958,7 +959,7 @@ public class MasterRpcServices extends RSRpcServices
         builder.setStartTime(result.getStartTime());
         builder.setLastUpdate(result.getLastUpdate());
         if (result.isFailed()) {
-          builder.setException(result.getForeignExceptionMessage());
+          
builder.setException(result.getForeignExceptionMessage().getForeignExchangeMessage());
         }
         if (result.hasResultData()) {
           builder.setResult(ByteStringer.wrap(result.getResult()));
@@ -1018,7 +1019,7 @@ public class MasterRpcServices extends RSRpcServices
       ListProceduresResponse.Builder response =
           ListProceduresResponse.newBuilder();
       for(ProcedureInfo p: master.listProcedures()) {
-        response.addProcedure(ProcedureInfo.convertToProcedureProto(p));
+        response.addProcedure(ProcedureUtil.convertToProcedureProto(p));
       }
       return response.build();
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
index 6a7e267..8a5eb35 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ProcedureSyncWait.java
@@ -83,7 +83,7 @@ public final class ProcedureSyncWait {
       if (result.isFailed()) {
         // If the procedure fails, we should always have an exception 
captured. Throw it.
         throw RemoteProcedureException.fromProto(
-          result.getForeignExceptionMessage()).unwrapRemoteException();
+          
result.getForeignExceptionMessage().getForeignExchangeMessage()).unwrapRemoteException();
       }
       return result.getResult();
     } else {

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResult.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResult.java 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResult.java
index 6c76ce4..6baf9e0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResult.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestResult.java
@@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.CellComparator;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -245,7 +244,7 @@ public class TestResult extends TestCase {
       LOG.debug("As expected: " + ex.getMessage());
     }
     try {
-      
emptyResult.addResults(ClientProtos.RegionLoadStats.getDefaultInstance());
+      emptyResult.addResults(new RegionLoadStats(0, 0, 0));
       fail("UnsupportedOperationException should have been thrown!");
     } catch (UnsupportedOperationException ex) {
       LOG.debug("As expected: " + ex.getMessage());

http://git-wip-us.apache.org/repos/asf/hbase/blob/97ad33c6/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
----------------------------------------------------------------------
diff --git 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
index 17b6e09..31190c1 100644
--- 
a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
+++ 
b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestProcedureAdmin.java
@@ -29,12 +29,12 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.ProcedureState;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
 import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.MediumTests;
-import 
org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;

Reply via email to