You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2016/10/07 19:14:56 UTC

[28/77] [abbrv] [partial] hbase git commit: HBASE-15638 Shade protobuf Which includes

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
index c2838ba..0572dcf 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureExecutor.java
@@ -50,7 +50,7 @@ import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureIterator
 import org.apache.hadoop.hbase.procedure2.util.StringUtils;
 import org.apache.hadoop.hbase.procedure2.util.TimeoutBlockingQueue;
 import org.apache.hadoop.hbase.procedure2.util.TimeoutBlockingQueue.TimeoutRetriever;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.hadoop.hbase.util.NonceKey;
@@ -593,7 +593,7 @@ public class ProcedureExecutor<TEnvironment> {
     List<ProcedureInfo> procedureLists =
         new ArrayList<ProcedureInfo>(procedures.size() + completed.size());
     for (java.util.Map.Entry<Long, Procedure> p: procedures.entrySet()) {
-      procedureLists.add(Procedure.createProcedureInfo(p.getValue(), null));
+      procedureLists.add(ProcedureUtil.createProcedureInfo(p.getValue()));
     }
     for (java.util.Map.Entry<Long, ProcedureInfo> e: completed.entrySet()) {
       // Note: The procedure could show up twice in the list with different state, as
@@ -1349,7 +1349,7 @@ public class ProcedureExecutor<TEnvironment> {
     execCompletionCleanup(proc);
 
     // update the executor internal state maps
-    ProcedureInfo procInfo = Procedure.createProcedureInfo(proc, proc.getNonceKey());
+    ProcedureInfo procInfo = ProcedureUtil.createProcedureInfo(proc, proc.getNonceKey());
     if (!proc.shouldWaitClientAck(getEnvironment())) {
       procInfo.setClientAckTime(0);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
new file mode 100644
index 0000000..05e8e09
--- /dev/null
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/ProcedureUtil.java
@@ -0,0 +1,108 @@
+/**
+ * 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.procedure2;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ProcedureInfo;
+import org.apache.hadoop.hbase.ProcedureState;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
+import org.apache.hadoop.hbase.util.NonceKey;
+
+/**
+ * Helper to convert to/from ProcedureProtos
+ */
+@InterfaceAudience.Private
+public final class ProcedureUtil {
+
+  private ProcedureUtil() { }
+
+  /**
+   * @return Convert the current {@link ProcedureInfo} into a Protocol Buffers Procedure
+   * instance.
+   */
+  public static ProcedureProtos.Procedure convertToProcedureProto(final ProcedureInfo procInfo) {
+    final 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(ForeignExceptionUtil.toProtoForeignException(procInfo.getException()));
+    }
+
+    if (procInfo.hasResultData()) {
+      builder.setResult(UnsafeByteOperations.unsafeWrap(procInfo.getResult()));
+    }
+
+    return builder.build();
+  }
+
+  /**
+   * Helper to convert the protobuf object.
+   * @return Convert the current Protocol Buffers Procedure to {@link ProcedureInfo}
+   * instance.
+   */
+  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() ?
+          ForeignExceptionUtil.toIOException(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 ProcedureInfo createProcedureInfo(final Procedure proc) {
+    return createProcedureInfo(proc, null);
+  }
+
+  /**
+   * Helper to create the ProcedureInfo from Procedure.
+   */
+  public static ProcedureInfo createProcedureInfo(final Procedure proc, final NonceKey nonceKey) {
+    final RemoteProcedureException exception = proc.hasException() ? proc.getException() : null;
+    return new ProcedureInfo(proc.getProcId(), proc.toStringClass(), proc.getOwner(),
+        convertToProcedureState(proc.getState()),
+        proc.hasParent() ? proc.getParentProcId() : -1, nonceKey,
+        exception != null ? exception.unwrapRemoteIOException() : null,
+        proc.getLastUpdate(), proc.getStartTime(), proc.getResult());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureException.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureException.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureException.java
index 402ddfc..5830c91 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureException.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RemoteProcedureException.java
@@ -22,11 +22,9 @@ import java.io.IOException;
 import org.apache.hadoop.ipc.RemoteException;
 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.shaded.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage;
 import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 
-import com.google.protobuf.InvalidProtocolBufferException;
-
 /**
  * A RemoteProcedureException is an exception from another thread or process.
  * <p>
@@ -106,8 +104,7 @@ public class RemoteProcedureException extends ProcedureException {
    * @return the ForeignExcpetion instance
    * @throws InvalidProtocolBufferException if there was deserialization problem this is thrown.
    */
-  public static RemoteProcedureException deserialize(byte[] bytes)
-      throws InvalidProtocolBufferException {
+  public static RemoteProcedureException deserialize(byte[] bytes) throws IOException {
     return fromProto(ForeignExceptionMessage.parseFrom(bytes));
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RootProcedureState.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RootProcedureState.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RootProcedureState.java
index b679cb1..2f118b7 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RootProcedureState.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/RootProcedureState.java
@@ -27,7 +27,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
 
 /**
  * Internal state of the ProcedureExecutor that describes the state of a "Root Procedure".

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
index f0bcdea..1a84070 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/SequentialProcedure.java
@@ -24,7 +24,7 @@ import java.io.OutputStream;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProcedureData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.SequentialProcedureData;
 
 /**
  * A SequentialProcedure describes one step in a procedure chain.

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
index a363c2e..2ec4418 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/StateMachineProcedure.java
@@ -29,7 +29,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.StateMachineProcedureData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.StateMachineProcedureData;
 
 /**
  * Procedure described by a series of steps.

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStoreTracker.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStoreTracker.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStoreTracker.java
index a60ba3f..4fea0d4 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStoreTracker.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/ProcedureStoreTracker.java
@@ -26,7 +26,7 @@ import java.util.TreeMap;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
 
 /**
  * Keeps track of live procedures.
@@ -278,7 +278,7 @@ public class ProcedureStoreTracker {
 
     /**
      * Convert to
-     * {@link org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode}
+     * org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureStoreTracker.TrackerNode
      * protobuf.
      */
     public ProcedureProtos.ProcedureStoreTracker.TrackerNode convert() {
@@ -768,7 +768,7 @@ public class ProcedureStoreTracker {
 
   /**
    * Builds
-   * {@link org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureStoreTracker}
+   * org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureStoreTracker
    * protocol buffer from current state.
    */
   public ProcedureProtos.ProcedureStoreTracker toProto() throws IOException {

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
index b9726a8..012ddeb 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFile.java
@@ -29,9 +29,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALTrailer;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALTrailer;
 
 /**
  * Describes a WAL File

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormat.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormat.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormat.java
index 5f726d0..e26e2d4 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormat.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALFormat.java
@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.hbase.procedure2.store.wal;
 
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -36,9 +36,9 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureLoader;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
 import org.apache.hadoop.hbase.procedure2.util.ByteSlot;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALEntry;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALTrailer;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALTrailer;
 
 /**
  * Helper class that contains the WAL serialization utils.

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/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 118ec19..fac9753 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
@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.hbase.procedure2.store.wal;
 
-import com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
 
 import java.io.IOException;
 
@@ -26,14 +26,14 @@ 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;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureIterator;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALEntry;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry;
 
 /**
  * Helper class that loads the procedures stored in a WAL

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALPrettyPrinter.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALPrettyPrinter.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALPrettyPrinter.java
index fb195b6..a6ec7dd 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALPrettyPrinter.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/ProcedureWALPrettyPrinter.java
@@ -39,8 +39,8 @@ 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.procedure2.Procedure;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALEntry;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALEntry;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
index bcd4e5f..36cf7af 100644
--- a/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
+++ b/hbase-procedure/src/main/java/org/apache/hadoop/hbase/procedure2/store/wal/WALProcedureStore.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hbase.procedure2.store.wal;
 
-import com.google.common.annotations.VisibleForTesting;
-
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -56,10 +54,12 @@ import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreBase;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStoreTracker;
 import org.apache.hadoop.hbase.procedure2.util.ByteSlot;
 import org.apache.hadoop.hbase.procedure2.util.StringUtils;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureWALHeader;
 import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.ipc.RemoteException;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * WAL implementation of the ProcedureStore.
  */

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/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 034109d..d767a0f 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
@@ -38,8 +38,7 @@ import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureIterator;
 import org.apache.hadoop.hbase.procedure2.store.NoopProcedureStore;
 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
-import org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.ForeignExceptionMessage;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
 import org.apache.hadoop.hbase.util.Threads;
 
 import static org.junit.Assert.assertEquals;
@@ -205,40 +204,35 @@ public class ProcedureTestingUtility {
   }
 
   public static void assertProcNotFailed(final ProcedureInfo result) {
-    if (result.getForeignExceptionMessage() != null) {
-      ForeignExceptionMessage exception =
-          result.getForeignExceptionMessage().getForeignExchangeMessage();
-      String msg = exception != null ? result.getExceptionFullMessage() : "no exception found";
-      assertFalse(msg, result.isFailed());
-    }
+    assertFalse("found exception: " + result.getException(), result.isFailed());
   }
 
   public static void assertIsAbortException(final ProcedureInfo result) {
     assertEquals(true, result.isFailed());
-    LOG.info(result.getExceptionFullMessage());
-    Throwable cause = getExceptionCause(result);
+    LOG.info(result.getException().getMessage());
+    Throwable cause = result.getException().getCause();
     assertTrue("expected abort exception, got "+ cause, cause instanceof ProcedureAbortedException);
   }
 
   public static void assertIsTimeoutException(final ProcedureInfo result) {
     assertEquals(true, result.isFailed());
-    LOG.info(result.getExceptionFullMessage());
-    Throwable cause = getExceptionCause(result);
+    LOG.info(result.getException().getMessage());
+    Throwable cause = result.getException();
     assertTrue("expected TimeoutIOException, got " + cause, cause instanceof TimeoutIOException);
   }
 
   public static void assertIsIllegalArgumentException(final ProcedureInfo result) {
     assertEquals(true, result.isFailed());
-    LOG.info(result.getExceptionFullMessage());
-    Throwable cause = ProcedureTestingUtility.getExceptionCause(result);
+    LOG.info(result.getException().getMessage());
+    Throwable cause = getExceptionCause(result);
     assertTrue("expected IllegalArgumentIOException, got " + cause,
       cause instanceof IllegalArgumentIOException);
   }
 
   public static Throwable getExceptionCause(final ProcedureInfo procInfo) {
-    assert procInfo.getForeignExceptionMessage() != null;
-    return RemoteProcedureException
-        .fromProto(procInfo.getForeignExceptionMessage().getForeignExchangeMessage()).getCause();
+    assert procInfo.isFailed();
+    Throwable cause = procInfo.getException().getCause();
+    return cause == null ? procInfo.getException() : cause;
   }
 
   public static class TestProcedure extends Procedure<Void> {

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestChildProcedures.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestChildProcedures.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestChildProcedures.java
index 61f2333..997999b 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestChildProcedures.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestChildProcedures.java
@@ -140,7 +140,7 @@ public class TestChildProcedures {
     assertTrue("expected completed proc", procExecutor.isFinished(procId));
     ProcedureInfo result = procExecutor.getResult(procId);
     assertEquals(true, result.isFailed());
-    LOG.info(result.getExceptionFullMessage());
+    LOG.info(result.getException().getMessage());
   }
 
   public static class TestRootProcedure extends SequentialProcedure<TestProcEnv> {

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureExecution.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureExecution.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureExecution.java
index b0cc43d..da6d960 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureExecution.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureExecution.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
 import org.apache.hadoop.hbase.ProcedureInfo;
 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
@@ -177,7 +177,7 @@ public class TestProcedureExecution {
     LOG.info(state);
     ProcedureInfo result = procExecutor.getResult(rootId);
     assertTrue(state.toString(), result.isFailed());
-    LOG.info(result.getExceptionFullMessage());
+    LOG.info(result.getException().getMessage());
     Throwable cause = ProcedureTestingUtility.getExceptionCause(result);
     assertTrue("expected TestProcedureException, got " + cause,
       cause instanceof TestProcedureException);
@@ -221,7 +221,7 @@ public class TestProcedureExecution {
 
     ProcedureInfo result = procExecutor.getResult(procId);
     assertTrue("expected a failure", result.isFailed());
-    LOG.info(result.getExceptionFullMessage());
+    LOG.info(result.getException().getMessage());
     Throwable cause = ProcedureTestingUtility.getExceptionCause(result);
     assertTrue("expected TestProcedureException, got " + cause,
       cause instanceof TestProcedureException);

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java
----------------------------------------------------------------------
diff --git a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java
index 5dea06e..408cffd 100644
--- a/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java
+++ b/hbase-procedure/src/test/java/org/apache/hadoop/hbase/procedure2/TestProcedureToString.java
@@ -23,8 +23,8 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 
-import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.ServerCrashState;
-import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.ProcedureState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ServerCrashState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
 import org.apache.hadoop.hbase.testclassification.MasterTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-protocol-shaded/README.txt
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/README.txt b/hbase-protocol-shaded/README.txt
new file mode 100644
index 0000000..d6f6ae2
--- /dev/null
+++ b/hbase-protocol-shaded/README.txt
@@ -0,0 +1,48 @@
+Please read carefully as the 'menu options' have changed.
+
+This module has proto files used by core. These protos
+overlap with protos that are used by coprocessor endpoints
+(CPEP) in the module hbase-protocol. So the core versions have
+a different name, the generated classes are relocated
+-- i.e. shaded -- to a new location; they are moved from
+org.apache.hadoop.hbase.* to org.apache.hadoop.hbase.shaded.
+
+This module also includes the protobuf that hbase core depends
+on again relocated to live at an offset of
+org.apache.hadoop.hbase.shaded so as to avoid clashes with other
+versions of protobuf resident on our CLASSPATH included,
+transitively or otherwise, by dependencies: i.e. the shaded
+protobuf Message class is at
+org.apache.hadoop.hbase.shaded.com.google.protobuf.Message
+rather than at com.google.protobuf.Message.
+
+Below we describe how to generate the java files for this
+module. Run this step any time you change the proto files
+in this module or if you change the protobuf version. If you
+add a new file, be sure to add mention of the proto in the
+pom.xml (scroll till you see the listing of protos to consider).
+
+First ensure that the appropriate protobuf protoc tool is in
+your $PATH as in:
+
+ $ export PATH=~/bin/protobuf-3.1.0/src:$PATH
+
+.. or pass -Dprotoc.path=PATH_TO_PROTOC when running
+the below mvn commands. You may need to download protobuf and
+build protoc first.
+
+Run:
+
+ $ mvn install -Dgenerate-shaded-classes
+
+or
+
+ $ mvn install -Pgenerate-shaded-classes
+
+to build and trigger the special generate-shaded-classes
+profile. When finished, the content of
+src/main/java/org/apache/hadoop/hbase/shaded will have
+been updated. Check in the changes.
+
+See the pom.xml under the generate-shaded-classes profile
+for more info on how this step works.

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-protocol-shaded/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/pom.xml b/hbase-protocol-shaded/pom.xml
new file mode 100644
index 0000000..e0e9eec
--- /dev/null
+++ b/hbase-protocol-shaded/pom.xml
@@ -0,0 +1,347 @@
+<?xml version="1.0"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <!--
+    /**
+     * 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.
+     */
+    -->
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>hbase</artifactId>
+    <groupId>org.apache.hbase</groupId>
+    <version>2.0.0-SNAPSHOT</version>
+    <relativePath>..</relativePath>
+  </parent>
+  <artifactId>hbase-protocol-shaded</artifactId>
+  <name>Apache HBase - Shaded Protocol</name>
+  <description>Shaded protobuf protocol classes used by HBase internally.</description>
+  <properties>
+    <maven.javadoc.skip>true</maven.javadoc.skip>
+    <!--Version of protobuf that hbase uses internally (we shade our pb)
+           -->
+    <internal.protobuf.version>3.1.0</internal.protobuf.version>
+    <!--The Default target dir-->
+    <classes.dir>${basedir}/target</classes.dir>
+    <!--The Default location for sources-->
+    <sources.dir>src/main/java</sources.dir>
+  </properties>
+  <build>
+    <!--I want to override these in profile so define them
+         with variables up here-->
+    <sourceDirectory>${sources.dir}</sourceDirectory>
+    <outputDirectory>${classes.dir}</outputDirectory>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-site-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+      <!-- Make a jar and put the sources in the jar -->
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-source-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <!--Make it so assembly:single does nothing in here-->
+        <artifactId>maven-assembly-plugin</artifactId>
+        <version>${maven.assembly.version}</version>
+        <configuration>
+          <skipAssembly>true</skipAssembly>
+        </configuration>
+      </plugin>
+      <plugin>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <!-- Always skip the second part executions, since we only run simple unit tests in this module -->
+        <executions>
+          <execution>
+            <id>secondPartTestsExecution</id>
+            <phase>test</phase>
+            <goals>
+              <goal>test</goal>
+            </goals>
+            <configuration>
+              <skip>true</skip>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+    <pluginManagement>
+      <plugins>
+        <!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
+        <plugin>
+          <groupId>org.eclipse.m2e</groupId>
+          <artifactId>lifecycle-mapping</artifactId>
+          <version>1.0.0</version>
+          <configuration>
+            <lifecycleMappingMetadata>
+              <pluginExecutions>
+                <pluginExecution>
+                  <pluginExecutionFilter>
+                    <groupId>org.apache.hadoop</groupId>
+                    <artifactId>hadoop-maven-plugins</artifactId>
+                    <versionRange>[2.0.5-alpha,)</versionRange>
+                    <goals>
+                      <goal>protoc</goal>
+                    </goals>
+                  </pluginExecutionFilter>
+                  <action>
+                    <ignore/>
+                  </action>
+                </pluginExecution>
+              </pluginExecutions>
+            </lifecycleMappingMetadata>
+          </configuration>
+        </plugin>
+      </plugins>
+    </pluginManagement>
+  </build>
+  <dependencies>
+    <!--BE CAREFUL! Any dependency added here needs to be
+          excluded above in the shade plugin else the dependency
+          will get bundled-->
+    <!-- Intra-project dependencies -->
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-annotations</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>jdk.tools</groupId>
+          <artifactId>jdk.tools</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <!-- General dependencies -->
+    <dependency>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java</artifactId>
+      <version>${internal.protobuf.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
+    </dependency>
+  </dependencies>
+  <profiles>
+    <!-- Skip the tests in this module -->
+    <profile>
+      <id>skip-protocol-shaded-tests</id>
+      <activation>
+        <property>
+          <name>skip-protocol-shaded-tests</name>
+        </property>
+      </activation>
+      <properties>
+        <surefire.skipFirstPart>true</surefire.skipFirstPart>
+      </properties>
+    </profile>
+    <!--
+         Generate shaded classes using proto files and
+         the protobuf lib we depend on. Drops generated
+         files under src/main/java. Check in the generated
+         files so available at build time.  Run this
+          profile/step everytime you change proto
+         files or update the protobuf version. If you add a
+         proto, be sure to add it to the list below in the
+         hadoop-maven-plugin else we won't 'see' it.
+
+         The below first generates java files from protos.
+         We then compile the generated files and make a jar
+         file.  The jar file is then passed to the shade plugin
+         which makes a new fat jar that includes the protobuf
+         lib but with all relocated given the
+         org.apache.hadoop.hbase.shaded prefix. The shading
+         step as by-product produces a jar with relocated
+         java source files in it. This jar we then unpack over
+         the src/main/java directory and we're done.
+
+         User is expected to check in the changes if they look
+         good.
+
+         TODO: Patch the protobuf lib using maven-patch-plugin
+         with changes we need.
+      -->
+    <profile>
+      <id>generate-shaded-classes</id>
+      <activation>
+        <property>
+          <name>generate-shaded-classes</name>
+        </property>
+      </activation>
+      <properties>
+        <profile.id>generate-shaded-classes</profile.id>
+        <sources.dir>${project.build.directory}/protoc-generated-sources</sources.dir>
+        <classes.dir>${project.build.directory}/protoc-generated-classes</classes.dir>
+        <!--When the compile for this profile runs, make sure it makes jars that
+             can be related back to this shading profile. Give them a shading prefix.
+         -->
+        <jar.finalName>${profile.id}.${artifactId}-${project.version}</jar.finalName>
+      </properties>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-maven-plugins</artifactId>
+            <configuration>
+              <protocVersion>${internal.protobuf.version}</protocVersion>
+            </configuration>
+            <executions>
+              <execution>
+                <id>compile-protoc</id>
+                <phase>generate-sources</phase>
+                <goals>
+                  <goal>protoc</goal>
+                </goals>
+                <configuration>
+                  <imports>
+                    <param>${basedir}/src/main/protobuf</param>
+                  </imports>
+                  <source>
+                    <directory>${basedir}/src/main/protobuf</directory>
+                    <!-- Unfortunately, Hadoop plugin does not support *.proto.
+                             We have to individually list every proto file here -->
+                    <includes>
+                      <include>Admin.proto</include>
+                      <include>Cell.proto</include>
+                      <include>Client.proto</include>
+                      <include>ClusterId.proto</include>
+                      <include>ClusterStatus.proto</include>
+                      <include>Comparator.proto</include>
+                      <include>Encryption.proto</include>
+                      <include>ErrorHandling.proto</include>
+                      <include>FS.proto</include>
+                      <include>Filter.proto</include>
+                      <include>HBase.proto</include>
+                      <include>HFile.proto</include>
+                      <include>LoadBalancer.proto</include>
+                      <include>MapReduce.proto</include>
+                      <include>Master.proto</include>
+                      <include>MasterProcedure.proto</include>
+                      <include>Procedure.proto</include>
+                      <include>Quota.proto</include>
+                      <include>RPC.proto</include>
+                      <include>RegionNormalizer.proto</include>
+                      <include>RegionServerStatus.proto</include>
+                      <include>Snapshot.proto</include>
+                      <include>Tracing.proto</include>
+                      <include>WAL.proto</include>
+                      <include>ZooKeeper.proto</include>
+                      <include>TestProcedure.proto</include>
+                      <include>test.proto</include>
+                      <include>test_rpc_service.proto</include>
+                    </includes>
+                  </source>
+                  <output>${sources.dir}</output>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-jar-plugin</artifactId>
+            <version>3.0.2</version>
+            <configuration>
+              <finalName>${jar.finalName}</finalName>                   
+            </configuration>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-shade-plugin</artifactId>
+            <version>2.4.3</version>
+            <executions>
+              <execution>
+                <phase>package</phase>
+                <goals>
+                  <goal>shade</goal>
+                </goals>
+                <configuration>
+                  <shadeSourcesContent>true</shadeSourcesContent>
+                  <createSourcesJar>true</createSourcesJar>
+                  <relocations>
+                    <relocation>
+                      <pattern>com.google.protobuf</pattern>
+                      <shadedPattern>org.apache.hadoop.hbase.shaded.com.google.protobuf</shadedPattern>
+                    </relocation>
+                  </relocations>
+                  <!-- What I got when I did a mvn dependency:list for this
+                   module. Exclude all but the protobuf
+                [INFO]    commons-logging:commons-logging:jar:1.2:compile
+                [INFO]    com.github.stephenc.findbugs:findbugs-annotations:jar:1.3.9-1:compile
+                [INFO]    log4j:log4j:jar:1.2.17:compile
+                [INFO]    com.google.protobuf:protobuf-java:jar:2.5.0:compile
+                [INFO]    org.hamcrest:hamcrest-core:jar:1.3:test
+                [INFO]    org.mockito:mockito-all:jar:1.10.8:test
+                [INFO]    junit:junit:jar:4.12:compile
+                [INFO]    org.apache.hbase:hbase-annotations:jar:2.0.0-SNAPSHOT:compile
+
+                  The list below must exlude all of the above except protobuf.
+              -->
+                  <artifactSet>
+                    <excludes>
+                      <exclude>commons-logging:commons-logging</exclude>
+                      <exclude>com.github.stephenc.findbugs:findbugs-annotations</exclude>
+                      <exclude>log4j:log4j</exclude>
+                      <exclude>org.hamcrest:hamcrest-core</exclude>
+                      <exclude>org.mockito:mockito-all</exclude>
+                      <exclude>junit:junit</exclude>
+                      <exclude>org.apache.hbase:hbase-annotations</exclude>
+                    </excludes>
+                  </artifactSet>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <!--Now unpack the shaded jar made above so the shaded classes
+             are available to subsequent modules-->
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-dependency-plugin</artifactId>
+            <version>2.10</version>
+            <executions>
+              <execution>
+                <id>unpack</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>unpack</goal>
+                </goals>
+                <configuration>
+                  <artifactItems>
+                    <artifactItem>
+                      <groupId>${project.groupId}</groupId>
+                      <artifactId>${project.artifactId}</artifactId>
+                      <version>${project.version}</version>
+                      <classifier>sources</classifier>
+                      <type>jar</type>
+                      <overWrite>true</overWrite>
+                      <outputDirectory>${default.sources.dir}</outputDirectory>
+                      <includes>**/*.java</includes>
+                    </artifactItem>
+                  </artifactItems>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <!--Patch the files here!!!
+               Use maven-patch-plugin
+               -->
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractMessage.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractMessage.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractMessage.java
new file mode 100644
index 0000000..4a6fefa
--- /dev/null
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractMessage.java
@@ -0,0 +1,646 @@
+// Protocol Buffers - Google's data interchange format
+// Copyright 2008 Google Inc.  All rights reserved.
+// https://developers.google.com/protocol-buffers/
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+package org.apache.hadoop.hbase.shaded.com.google.protobuf;
+
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.EnumValueDescriptor;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.FieldDescriptor;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Descriptors.OneofDescriptor;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Internal.EnumLite;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A partial implementation of the {@link Message} interface which implements
+ * as many methods of that interface as possible in terms of other methods.
+ *
+ * @author kenton@google.com Kenton Varda
+ */
+public abstract class AbstractMessage
+    // TODO(dweis): Update GeneratedMessage to parameterize with MessageType and BuilderType.
+    extends AbstractMessageLite
+    implements Message {
+
+  @Override
+  public boolean isInitialized() {
+    return MessageReflection.isInitialized(this);
+  }
+
+  /**
+   * Interface for the parent of a Builder that allows the builder to
+   * communicate invalidations back to the parent for use when using nested
+   * builders.
+   */
+  protected interface BuilderParent {
+
+    /**
+     * A builder becomes dirty whenever a field is modified -- including fields
+     * in nested builders -- and becomes clean when build() is called.  Thus,
+     * when a builder becomes dirty, all its parents become dirty as well, and
+     * when it becomes clean, all its children become clean.  The dirtiness
+     * state is used to invalidate certain cached values.
+     * <br>
+     * To this end, a builder calls markDirty() on its parent whenever it
+     * transitions from clean to dirty.  The parent must propagate this call to
+     * its own parent, unless it was already dirty, in which case the
+     * grandparent must necessarily already be dirty as well.  The parent can
+     * only transition back to "clean" after calling build() on all children.
+     */
+    void markDirty();
+  }
+
+  /** Create a nested builder. */
+  protected Message.Builder newBuilderForType(BuilderParent parent) {
+    throw new UnsupportedOperationException("Nested builder is not supported for this type.");
+  }
+
+
+  @Override
+  public List<String> findInitializationErrors() {
+    return MessageReflection.findMissingFields(this);
+  }
+
+  @Override
+  public String getInitializationErrorString() {
+    return MessageReflection.delimitWithCommas(findInitializationErrors());
+  }
+
+  /** TODO(jieluo): Clear it when all subclasses have implemented this method. */
+  @Override
+  public boolean hasOneof(OneofDescriptor oneof) {
+    throw new UnsupportedOperationException("hasOneof() is not implemented.");
+  }
+
+  /** TODO(jieluo): Clear it when all subclasses have implemented this method. */
+  @Override
+  public FieldDescriptor getOneofFieldDescriptor(OneofDescriptor oneof) {
+    throw new UnsupportedOperationException(
+        "getOneofFieldDescriptor() is not implemented.");
+  }
+
+  @Override
+  public final String toString() {
+    return TextFormat.printToString(this);
+  }
+
+  @Override
+  public void writeTo(final CodedOutputStream output) throws IOException {
+    MessageReflection.writeMessageTo(this, getAllFields(), output, false);
+  }
+
+  protected int memoizedSize = -1;
+
+  @Override
+  public int getSerializedSize() {
+    int size = memoizedSize;
+    if (size != -1) {
+      return size;
+    }
+
+    memoizedSize = MessageReflection.getSerializedSize(this, getAllFields());
+    return memoizedSize;
+  }
+
+  @Override
+  public boolean equals(final Object other) {
+    if (other == this) {
+      return true;
+    }
+    if (!(other instanceof Message)) {
+      return false;
+    }
+    final Message otherMessage = (Message) other;
+    if (getDescriptorForType() != otherMessage.getDescriptorForType()) {
+      return false;
+    }
+    return compareFields(getAllFields(), otherMessage.getAllFields()) &&
+        getUnknownFields().equals(otherMessage.getUnknownFields());
+  }
+
+  @Override
+  public int hashCode() {
+    int hash = memoizedHashCode;
+    if (hash == 0) {
+      hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      hash = hashFields(hash, getAllFields());
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+    }
+    return hash;
+  }
+  
+  private static ByteString toByteString(Object value) {
+    if (value instanceof byte[]) {
+      return ByteString.copyFrom((byte[]) value);
+    } else {
+      return (ByteString) value;
+    }
+  }
+ 
+  /**
+   * Compares two bytes fields. The parameters must be either a byte array or a
+   * ByteString object. They can be of different type though.
+   */
+  private static boolean compareBytes(Object a, Object b) {
+    if (a instanceof byte[] && b instanceof byte[]) {
+      return Arrays.equals((byte[])a, (byte[])b);
+    }
+    return toByteString(a).equals(toByteString(b));
+  }
+  
+  /**
+   * Converts a list of MapEntry messages into a Map used for equals() and
+   * hashCode().
+   */
+  @SuppressWarnings({"rawtypes", "unchecked"})
+  private static Map convertMapEntryListToMap(List list) {
+    if (list.isEmpty()) {
+      return Collections.emptyMap();
+    }
+    Map result = new HashMap();
+    Iterator iterator = list.iterator();
+    Message entry = (Message) iterator.next();
+    Descriptors.Descriptor descriptor = entry.getDescriptorForType();
+    Descriptors.FieldDescriptor key = descriptor.findFieldByName("key");
+    Descriptors.FieldDescriptor value = descriptor.findFieldByName("value");
+    Object fieldValue = entry.getField(value);
+    if (fieldValue instanceof EnumValueDescriptor) {
+      fieldValue = ((EnumValueDescriptor) fieldValue).getNumber();
+    }
+    result.put(entry.getField(key), fieldValue);
+    while (iterator.hasNext()) {
+      entry = (Message) iterator.next();
+      fieldValue = entry.getField(value);
+      if (fieldValue instanceof EnumValueDescriptor) {
+        fieldValue = ((EnumValueDescriptor) fieldValue).getNumber();
+      }
+      result.put(entry.getField(key), fieldValue);
+    }
+    return result;
+  }
+  
+  /**
+   * Compares two map fields. The parameters must be a list of MapEntry
+   * messages.
+   */
+  @SuppressWarnings({"rawtypes", "unchecked"})
+  private static boolean compareMapField(Object a, Object b) {
+    Map ma = convertMapEntryListToMap((List) a);
+    Map mb = convertMapEntryListToMap((List) b);
+    return MapFieldLite.equals(ma, mb);
+  }
+  
+  /**
+   * Compares two set of fields.
+   * This method is used to implement {@link AbstractMessage#equals(Object)}
+   * and {@link AbstractMutableMessage#equals(Object)}. It takes special care
+   * of bytes fields because immutable messages and mutable messages use
+   * different Java type to reprensent a bytes field and this method should be
+   * able to compare immutable messages, mutable messages and also an immutable
+   * message to a mutable message.
+   */
+  static boolean compareFields(Map<FieldDescriptor, Object> a,
+      Map<FieldDescriptor, Object> b) {
+    if (a.size() != b.size()) {
+      return false;
+    }
+    for (FieldDescriptor descriptor : a.keySet()) {
+      if (!b.containsKey(descriptor)) {
+        return false;
+      }
+      Object value1 = a.get(descriptor);
+      Object value2 = b.get(descriptor);
+      if (descriptor.getType() == FieldDescriptor.Type.BYTES) {
+        if (descriptor.isRepeated()) {
+          List list1 = (List) value1;
+          List list2 = (List) value2;
+          if (list1.size() != list2.size()) {
+            return false;
+          }
+          for (int i = 0; i < list1.size(); i++) {
+            if (!compareBytes(list1.get(i), list2.get(i))) {
+              return false;
+            }
+          }
+        } else {
+          // Compares a singular bytes field.
+          if (!compareBytes(value1, value2)) {
+            return false;
+          }
+        }
+      } else if (descriptor.isMapField()) {
+        if (!compareMapField(value1, value2)) {
+          return false;
+        }
+      } else {
+        // Compare non-bytes fields.
+        if (!value1.equals(value2)) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+  
+  /**
+   * Calculates the hash code of a map field. {@code value} must be a list of
+   * MapEntry messages.
+   */
+  @SuppressWarnings("unchecked")
+  private static int hashMapField(Object value) {
+    return MapFieldLite.calculateHashCodeForMap(convertMapEntryListToMap((List) value));
+  }
+
+  /** Get a hash code for given fields and values, using the given seed. */
+  @SuppressWarnings("unchecked")
+  protected static int hashFields(int hash, Map<FieldDescriptor, Object> map) {
+    for (Map.Entry<FieldDescriptor, Object> entry : map.entrySet()) {
+      FieldDescriptor field = entry.getKey();
+      Object value = entry.getValue();
+      hash = (37 * hash) + field.getNumber();
+      if (field.isMapField()) {
+        hash = (53 * hash) + hashMapField(value);
+      } else if (field.getType() != FieldDescriptor.Type.ENUM){
+        hash = (53 * hash) + value.hashCode();
+      } else if (field.isRepeated()) {
+        List<? extends EnumLite> list = (List<? extends EnumLite>) value;
+        hash = (53 * hash) + Internal.hashEnumList(list);
+      } else {
+        hash = (53 * hash) + Internal.hashEnum((EnumLite) value);
+      }
+    }
+    return hash;
+  }
+
+  /**
+   * Package private helper method for AbstractParser to create
+   * UninitializedMessageException with missing field information.
+   */
+  @Override
+  UninitializedMessageException newUninitializedMessageException() {
+    return Builder.newUninitializedMessageException(this);
+  }
+
+  // =================================================================
+
+  /**
+   * A partial implementation of the {@link Message.Builder} interface which
+   * implements as many methods of that interface as possible in terms of
+   * other methods.
+   */
+  @SuppressWarnings("unchecked")
+  public static abstract class Builder<BuilderType extends Builder<BuilderType>>
+      extends AbstractMessageLite.Builder
+      implements Message.Builder {
+    // The compiler produces an error if this is not declared explicitly.
+    @Override
+    public abstract BuilderType clone();
+
+    /** TODO(jieluo): Clear it when all subclasses have implemented this method. */
+    @Override
+    public boolean hasOneof(OneofDescriptor oneof) {
+      throw new UnsupportedOperationException("hasOneof() is not implemented.");
+    }
+
+    /** TODO(jieluo): Clear it when all subclasses have implemented this method. */
+    @Override
+    public FieldDescriptor getOneofFieldDescriptor(OneofDescriptor oneof) {
+      throw new UnsupportedOperationException(
+          "getOneofFieldDescriptor() is not implemented.");
+    }
+
+    /** TODO(jieluo): Clear it when all subclasses have implemented this method. */
+    @Override
+    public BuilderType clearOneof(OneofDescriptor oneof) {
+      throw new UnsupportedOperationException("clearOneof() is not implemented.");
+    }
+
+    @Override
+    public BuilderType clear() {
+      for (final Map.Entry<FieldDescriptor, Object> entry :
+           getAllFields().entrySet()) {
+        clearField(entry.getKey());
+      }
+      return (BuilderType) this;
+    }
+
+    @Override
+    public List<String> findInitializationErrors() {
+      return MessageReflection.findMissingFields(this);
+    }
+
+    @Override
+    public String getInitializationErrorString() {
+      return MessageReflection.delimitWithCommas(findInitializationErrors());
+    }
+    
+    @Override
+    protected BuilderType internalMergeFrom(AbstractMessageLite other) {
+      return mergeFrom((Message) other);
+    }
+
+    @Override
+    public BuilderType mergeFrom(final Message other) {
+      if (other.getDescriptorForType() != getDescriptorForType()) {
+        throw new IllegalArgumentException(
+          "mergeFrom(Message) can only merge messages of the same type.");
+      }
+
+      // Note:  We don't attempt to verify that other's fields have valid
+      //   types.  Doing so would be a losing battle.  We'd have to verify
+      //   all sub-messages as well, and we'd have to make copies of all of
+      //   them to insure that they don't change after verification (since
+      //   the Message interface itself cannot enforce immutability of
+      //   implementations).
+      // TODO(kenton):  Provide a function somewhere called makeDeepCopy()
+      //   which allows people to make secure deep copies of messages.
+
+      for (final Map.Entry<FieldDescriptor, Object> entry :
+           other.getAllFields().entrySet()) {
+        final FieldDescriptor field = entry.getKey();
+        if (field.isRepeated()) {
+          for (final Object element : (List)entry.getValue()) {
+            addRepeatedField(field, element);
+          }
+        } else if (field.getJavaType() == FieldDescriptor.JavaType.MESSAGE) {
+          final Message existingValue = (Message)getField(field);
+          if (existingValue == existingValue.getDefaultInstanceForType()) {
+            setField(field, entry.getValue());
+          } else {
+            setField(field,
+              existingValue.newBuilderForType()
+                .mergeFrom(existingValue)
+                .mergeFrom((Message)entry.getValue())
+                .build());
+          }
+        } else {
+          setField(field, entry.getValue());
+        }
+      }
+
+      mergeUnknownFields(other.getUnknownFields());
+
+      return (BuilderType) this;
+    }
+
+    @Override
+    public BuilderType mergeFrom(final CodedInputStream input)
+                                 throws IOException {
+      return mergeFrom(input, ExtensionRegistry.getEmptyRegistry());
+    }
+
+    @Override
+    public BuilderType mergeFrom(
+        final CodedInputStream input,
+        final ExtensionRegistryLite extensionRegistry)
+        throws IOException {
+      final UnknownFieldSet.Builder unknownFields =
+        UnknownFieldSet.newBuilder(getUnknownFields());
+      while (true) {
+        final int tag = input.readTag();
+        if (tag == 0) {
+          break;
+        }
+
+        MessageReflection.BuilderAdapter builderAdapter =
+            new MessageReflection.BuilderAdapter(this);
+        if (!MessageReflection.mergeFieldFrom(input, unknownFields,
+                                              extensionRegistry,
+                                              getDescriptorForType(),
+                                              builderAdapter,
+                                              tag)) {
+          // end group tag
+          break;
+        }
+      }
+      setUnknownFields(unknownFields.build());
+      return (BuilderType) this;
+    }
+
+    @Override
+    public BuilderType mergeUnknownFields(final UnknownFieldSet unknownFields) {
+      setUnknownFields(
+        UnknownFieldSet.newBuilder(getUnknownFields())
+                       .mergeFrom(unknownFields)
+                       .build());
+      return (BuilderType) this;
+    }
+
+    @Override
+    public Message.Builder getFieldBuilder(final FieldDescriptor field) {
+      throw new UnsupportedOperationException(
+          "getFieldBuilder() called on an unsupported message type.");
+    }
+
+    @Override
+    public Message.Builder getRepeatedFieldBuilder(final FieldDescriptor field, int index) {
+      throw new UnsupportedOperationException(
+          "getRepeatedFieldBuilder() called on an unsupported message type.");
+    }
+
+    @Override
+    public String toString() {
+      return TextFormat.printToString(this);
+    }
+
+    /**
+     * Construct an UninitializedMessageException reporting missing fields in
+     * the given message.
+     */
+    protected static UninitializedMessageException
+        newUninitializedMessageException(Message message) {
+      return new UninitializedMessageException(
+          MessageReflection.findMissingFields(message));
+    }
+
+    /**
+     * Used to support nested builders and called to mark this builder as clean.
+     * Clean builders will propagate the {@link BuilderParent#markDirty()} event
+     * to their parent builders, while dirty builders will not, as their parents
+     * should be dirty already.
+     *
+     * NOTE: Implementations that don't support nested builders don't need to
+     * override this method.
+     */
+    void markClean() {
+      throw new IllegalStateException("Should be overridden by subclasses.");
+    }
+
+    /**
+     * Used to support nested builders and called when this nested builder is
+     * no longer used by its parent builder and should release the reference
+     * to its parent builder.
+     *
+     * NOTE: Implementations that don't support nested builders don't need to
+     * override this method.
+     */
+    void dispose() {
+      throw new IllegalStateException("Should be overridden by subclasses.");
+    }
+
+    // ===============================================================
+    // The following definitions seem to be required in order to make javac
+    // not produce weird errors like:
+    //
+    // java/org.apache.hadoop.hbase.shaded.com.google.protobuf/DynamicMessage.java:203: types
+    //   org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessage.Builder<
+    //     org.apache.hadoop.hbase.shaded.com.google.protobuf.DynamicMessage.Builder> and
+    //   org.apache.hadoop.hbase.shaded.com.google.protobuf.AbstractMessage.Builder<
+    //     org.apache.hadoop.hbase.shaded.com.google.protobuf.DynamicMessage.Builder> are incompatible; both
+    //   define mergeFrom(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString), but with unrelated
+    //   return types.
+    //
+    // Strangely, these lines are only needed if javac is invoked separately
+    // on AbstractMessage.java and AbstractMessageLite.java.  If javac is
+    // invoked on both simultaneously, it works.  (Or maybe the important
+    // point is whether or not DynamicMessage.java is compiled together with
+    // AbstractMessageLite.java -- not sure.)  I suspect this is a compiler
+    // bug.
+
+    @Override
+    public BuilderType mergeFrom(final ByteString data)
+        throws InvalidProtocolBufferException {
+      return (BuilderType) super.mergeFrom(data);
+    }
+
+    @Override
+    public BuilderType mergeFrom(
+        final ByteString data,
+        final ExtensionRegistryLite extensionRegistry)
+        throws InvalidProtocolBufferException {
+      return (BuilderType) super.mergeFrom(data, extensionRegistry);
+    }
+
+    @Override
+    public BuilderType mergeFrom(final byte[] data)
+        throws InvalidProtocolBufferException {
+      return (BuilderType) super.mergeFrom(data);
+    }
+
+    @Override
+    public BuilderType mergeFrom(
+        final byte[] data, final int off, final int len)
+        throws InvalidProtocolBufferException {
+      return (BuilderType) super.mergeFrom(data, off, len);
+    }
+
+    @Override
+    public BuilderType mergeFrom(
+        final byte[] data,
+        final ExtensionRegistryLite extensionRegistry)
+        throws InvalidProtocolBufferException {
+      return (BuilderType) super.mergeFrom(data, extensionRegistry);
+    }
+
+    @Override
+    public BuilderType mergeFrom(
+        final byte[] data, final int off, final int len,
+        final ExtensionRegistryLite extensionRegistry)
+        throws InvalidProtocolBufferException {
+      return (BuilderType) super.mergeFrom(data, off, len, extensionRegistry);
+    }
+
+    @Override
+    public BuilderType mergeFrom(final InputStream input)
+        throws IOException {
+      return (BuilderType) super.mergeFrom(input);
+    }
+
+    @Override
+    public BuilderType mergeFrom(
+        final InputStream input,
+        final ExtensionRegistryLite extensionRegistry)
+        throws IOException {
+      return (BuilderType) super.mergeFrom(input, extensionRegistry);
+    }
+
+    @Override
+    public boolean mergeDelimitedFrom(final InputStream input)
+        throws IOException {
+      return super.mergeDelimitedFrom(input);
+    }
+
+    @Override
+    public boolean mergeDelimitedFrom(
+        final InputStream input,
+        final ExtensionRegistryLite extensionRegistry)
+        throws IOException {
+      return super.mergeDelimitedFrom(input, extensionRegistry);
+    }
+  }
+
+  /**
+   * @deprecated from v3.0.0-beta-3+, for compatibility with v2.5.0 and v2.6.1
+   * generated code.
+   */
+  @Deprecated
+  protected static int hashLong(long n) {
+    return (int) (n ^ (n >>> 32));
+  }
+  //
+  /**
+   * @deprecated from v3.0.0-beta-3+, for compatibility with v2.5.0 and v2.6.1
+   * generated code.
+   */
+  @Deprecated
+  protected static int hashBoolean(boolean b) {
+    return b ? 1231 : 1237;
+  }
+  //
+  /**
+   * @deprecated from v3.0.0-beta-3+, for compatibility with v2.5.0 and v2.6.1
+   * generated code.
+   */
+  @Deprecated
+  protected static int hashEnum(EnumLite e) {
+    return e.getNumber();
+  }
+  //
+  /**
+   * @deprecated from v3.0.0-beta-3+, for compatibility with v2.5.0 and v2.6.1
+   * generated code.
+   */
+  @Deprecated
+  protected static int hashEnumList(List<? extends EnumLite> list) {
+    int hash = 1;
+    for (EnumLite e : list) {
+      hash = 31 * hash + hashEnum(e);
+    }
+    return hash;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractMessageLite.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractMessageLite.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractMessageLite.java
new file mode 100644
index 0000000..c294f10
--- /dev/null
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/com/google/protobuf/AbstractMessageLite.java
@@ -0,0 +1,384 @@
+// Protocol Buffers - Google's data interchange format
+// Copyright 2008 Google Inc.  All rights reserved.
+// https://developers.google.com/protocol-buffers/
+//
+// Redistribution and use in source and binary forms, with or without
+// modification, are permitted provided that the following conditions are
+// met:
+//
+//     * Redistributions of source code must retain the above copyright
+// notice, this list of conditions and the following disclaimer.
+//     * Redistributions in binary form must reproduce the above
+// copyright notice, this list of conditions and the following disclaimer
+// in the documentation and/or other materials provided with the
+// distribution.
+//     * Neither the name of Google Inc. nor the names of its
+// contributors may be used to endorse or promote products derived from
+// this software without specific prior written permission.
+//
+// THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+// "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+// LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+// A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+// OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+// SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+// LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+// DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+// THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+// (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+// OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+package org.apache.hadoop.hbase.shaded.com.google.protobuf;
+
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Collection;
+
+/**
+ * A partial implementation of the {@link MessageLite} interface which
+ * implements as many methods of that interface as possible in terms of other
+ * methods.
+ *
+ * @author kenton@google.com Kenton Varda
+ */
+public abstract class AbstractMessageLite<
+    MessageType extends AbstractMessageLite<MessageType, BuilderType>,
+    BuilderType extends AbstractMessageLite.Builder<MessageType, BuilderType>>
+        implements MessageLite {
+  protected int memoizedHashCode = 0;
+
+  @Override
+  public ByteString toByteString() {
+    try {
+      final ByteString.CodedBuilder out =
+        ByteString.newCodedBuilder(getSerializedSize());
+      writeTo(out.getCodedOutput());
+      return out.build();
+    } catch (IOException e) {
+      throw new RuntimeException(getSerializingExceptionMessage("ByteString"), e);
+    }
+  }
+
+  @Override
+  public byte[] toByteArray() {
+    try {
+      final byte[] result = new byte[getSerializedSize()];
+      final CodedOutputStream output = CodedOutputStream.newInstance(result);
+      writeTo(output);
+      output.checkNoSpaceLeft();
+      return result;
+    } catch (IOException e) {
+      throw new RuntimeException(getSerializingExceptionMessage("byte array"), e);
+    }
+  }
+
+  @Override
+  public void writeTo(final OutputStream output) throws IOException {
+    final int bufferSize =
+        CodedOutputStream.computePreferredBufferSize(getSerializedSize());
+    final CodedOutputStream codedOutput =
+        CodedOutputStream.newInstance(output, bufferSize);
+    writeTo(codedOutput);
+    codedOutput.flush();
+  }
+
+  @Override
+  public void writeDelimitedTo(final OutputStream output) throws IOException {
+    final int serialized = getSerializedSize();
+    final int bufferSize = CodedOutputStream.computePreferredBufferSize(
+        CodedOutputStream.computeRawVarint32Size(serialized) + serialized);
+    final CodedOutputStream codedOutput =
+        CodedOutputStream.newInstance(output, bufferSize);
+    codedOutput.writeRawVarint32(serialized);
+    writeTo(codedOutput);
+    codedOutput.flush();
+  }
+
+
+  /**
+   * Package private helper method for AbstractParser to create
+   * UninitializedMessageException.
+   */
+  UninitializedMessageException newUninitializedMessageException() {
+    return new UninitializedMessageException(this);
+  }
+
+  private String getSerializingExceptionMessage(String target) {
+    return "Serializing " + getClass().getName() + " to a " + target
+        + " threw an IOException (should never happen).";
+  }
+
+  protected static void checkByteStringIsUtf8(ByteString byteString)
+      throws IllegalArgumentException {
+    if (!byteString.isValidUtf8()) {
+      throw new IllegalArgumentException("Byte string is not UTF-8.");
+    }
+  }
+
+  protected static <T> void addAll(final Iterable<T> values,
+      final Collection<? super T> list) {
+    Builder.addAll(values, list);
+  }
+
+  /**
+   * A partial implementation of the {@link Message.Builder} interface which
+   * implements as many methods of that interface as possible in terms of
+   * other methods.
+   */
+  @SuppressWarnings("unchecked")
+  public abstract static class Builder<
+      MessageType extends AbstractMessageLite<MessageType, BuilderType>,
+      BuilderType extends Builder<MessageType, BuilderType>>
+      implements MessageLite.Builder {
+    // The compiler produces an error if this is not declared explicitly.
+    @Override
+    public abstract BuilderType clone();
+
+    @Override
+    public BuilderType mergeFrom(final CodedInputStream input) throws IOException {
+      return mergeFrom(input, ExtensionRegistryLite.getEmptyRegistry());
+    }
+
+    // Re-defined here for return type covariance.
+    @Override
+    public abstract BuilderType mergeFrom(
+        final CodedInputStream input, final ExtensionRegistryLite extensionRegistry)
+        throws IOException;
+
+    @Override
+    public BuilderType mergeFrom(final ByteString data) throws InvalidProtocolBufferException {
+      try {
+        final CodedInputStream input = data.newCodedInput();
+        mergeFrom(input);
+        input.checkLastTagWas(0);
+        return (BuilderType) this;
+      } catch (InvalidProtocolBufferException e) {
+        throw e;
+      } catch (IOException e) {
+        throw new RuntimeException(getReadingExceptionMessage("ByteString"), e);
+      }
+    }
+
+    @Override
+    public BuilderType mergeFrom(
+        final ByteString data, final ExtensionRegistryLite extensionRegistry)
+        throws InvalidProtocolBufferException {
+      try {
+        final CodedInputStream input = data.newCodedInput();
+        mergeFrom(input, extensionRegistry);
+        input.checkLastTagWas(0);
+        return (BuilderType) this;
+      } catch (InvalidProtocolBufferException e) {
+        throw e;
+      } catch (IOException e) {
+        throw new RuntimeException(getReadingExceptionMessage("ByteString"), e);
+      }
+    }
+
+    @Override
+    public BuilderType mergeFrom(final byte[] data) throws InvalidProtocolBufferException {
+      return mergeFrom(data, 0, data.length);
+    }
+
+    @Override
+    public BuilderType mergeFrom(final byte[] data, final int off, final int len)
+        throws InvalidProtocolBufferException {
+      try {
+        final CodedInputStream input =
+            CodedInputStream.newInstance(data, off, len);
+        mergeFrom(input);
+        input.checkLastTagWas(0);
+        return (BuilderType) this;
+      } catch (InvalidProtocolBufferException e) {
+        throw e;
+      } catch (IOException e) {
+        throw new RuntimeException(getReadingExceptionMessage("byte array"), e);
+      }
+    }
+
+    @Override
+    public BuilderType mergeFrom(final byte[] data, final ExtensionRegistryLite extensionRegistry)
+        throws InvalidProtocolBufferException {
+      return mergeFrom(data, 0, data.length, extensionRegistry);
+    }
+
+    @Override
+    public BuilderType mergeFrom(
+        final byte[] data,
+        final int off,
+        final int len,
+        final ExtensionRegistryLite extensionRegistry)
+        throws InvalidProtocolBufferException {
+      try {
+        final CodedInputStream input =
+            CodedInputStream.newInstance(data, off, len);
+        mergeFrom(input, extensionRegistry);
+        input.checkLastTagWas(0);
+        return (BuilderType) this;
+      } catch (InvalidProtocolBufferException e) {
+        throw e;
+      } catch (IOException e) {
+        throw new RuntimeException(getReadingExceptionMessage("byte array"), e);
+      }
+    }
+
+    @Override
+    public BuilderType mergeFrom(final InputStream input) throws IOException {
+      final CodedInputStream codedInput = CodedInputStream.newInstance(input);
+      mergeFrom(codedInput);
+      codedInput.checkLastTagWas(0);
+      return (BuilderType) this;
+    }
+
+    @Override
+    public BuilderType mergeFrom(
+        final InputStream input, final ExtensionRegistryLite extensionRegistry) throws IOException {
+      final CodedInputStream codedInput = CodedInputStream.newInstance(input);
+      mergeFrom(codedInput, extensionRegistry);
+      codedInput.checkLastTagWas(0);
+      return (BuilderType) this;
+    }
+
+    /**
+     * An InputStream implementations which reads from some other InputStream
+     * but is limited to a particular number of bytes.  Used by
+     * mergeDelimitedFrom().  This is intentionally package-private so that
+     * UnknownFieldSet can share it.
+     */
+    static final class LimitedInputStream extends FilterInputStream {
+      private int limit;
+
+      LimitedInputStream(InputStream in, int limit) {
+        super(in);
+        this.limit = limit;
+      }
+
+      @Override
+      public int available() throws IOException {
+        return Math.min(super.available(), limit);
+      }
+
+      @Override
+      public int read() throws IOException {
+        if (limit <= 0) {
+          return -1;
+        }
+        final int result = super.read();
+        if (result >= 0) {
+          --limit;
+        }
+        return result;
+      }
+
+      @Override
+      public int read(final byte[] b, final int off, int len)
+                      throws IOException {
+        if (limit <= 0) {
+          return -1;
+        }
+        len = Math.min(len, limit);
+        final int result = super.read(b, off, len);
+        if (result >= 0) {
+          limit -= result;
+        }
+        return result;
+      }
+
+      @Override
+      public long skip(final long n) throws IOException {
+        final long result = super.skip(Math.min(n, limit));
+        if (result >= 0) {
+          limit -= result;
+        }
+        return result;
+      }
+    }
+
+    @Override
+    public boolean mergeDelimitedFrom(
+        final InputStream input, final ExtensionRegistryLite extensionRegistry) throws IOException {
+      final int firstByte = input.read();
+      if (firstByte == -1) {
+        return false;
+      }
+      final int size = CodedInputStream.readRawVarint32(firstByte, input);
+      final InputStream limitedInput = new LimitedInputStream(input, size);
+      mergeFrom(limitedInput, extensionRegistry);
+      return true;
+    }
+
+    @Override
+    public boolean mergeDelimitedFrom(final InputStream input) throws IOException {
+      return mergeDelimitedFrom(input,
+          ExtensionRegistryLite.getEmptyRegistry());
+    }
+
+    @Override
+    @SuppressWarnings("unchecked") // isInstance takes care of this
+    public BuilderType mergeFrom(final MessageLite other) {
+      if (!getDefaultInstanceForType().getClass().isInstance(other)) {
+        throw new IllegalArgumentException(
+            "mergeFrom(MessageLite) can only merge messages of the same type.");
+      }
+
+      return internalMergeFrom((MessageType) other);
+    }
+
+    protected abstract BuilderType internalMergeFrom(MessageType message);
+
+    private String getReadingExceptionMessage(String target) {
+      return "Reading " + getClass().getName() + " from a " + target
+          + " threw an IOException (should never happen).";
+    }
+
+    /**
+     * Construct an UninitializedMessageException reporting missing fields in
+     * the given message.
+     */
+    protected static UninitializedMessageException
+        newUninitializedMessageException(MessageLite message) {
+      return new UninitializedMessageException(message);
+    }
+
+    /**
+     * Adds the {@code values} to the {@code list}.  This is a helper method
+     * used by generated code.  Users should ignore it.
+     *
+     * @throws NullPointerException if {@code values} or any of the elements of
+     * {@code values} is null. When that happens, some elements of
+     * {@code values} may have already been added to the result {@code list}.
+     */
+    protected static <T> void addAll(final Iterable<T> values,
+                                     final Collection<? super T> list) {
+      if (values == null) {
+        throw new NullPointerException();
+      }
+      if (values instanceof LazyStringList) {
+        // For StringOrByteStringLists, check the underlying elements to avoid
+        // forcing conversions of ByteStrings to Strings.
+        checkForNullValues(((LazyStringList) values).getUnderlyingElements());
+        list.addAll((Collection<T>) values);
+      } else if (values instanceof Collection) {
+        checkForNullValues(values);
+        list.addAll((Collection<T>) values);
+      } else {
+        for (final T value : values) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          list.add(value);
+        }
+      }
+    }
+
+    private static void checkForNullValues(final Iterable<?> values) {
+      for (final Object value : values) {
+        if (value == null) {
+          throw new NullPointerException();
+        }
+      }
+    }
+  }
+}