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:15:15 UTC

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
index 8a85580..ce5adda 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcConnection.java
@@ -22,7 +22,7 @@ import static org.apache.hadoop.hbase.ipc.CallEvent.Type.TIMEOUT;
 import static org.apache.hadoop.hbase.ipc.IPCUtil.setCancelled;
 import static org.apache.hadoop.hbase.ipc.IPCUtil.toIOE;
 
-import com.google.protobuf.RpcCallback;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcCallback;
 
 import io.netty.bootstrap.Bootstrap;
 import io.netty.buffer.ByteBuf;
@@ -52,7 +52,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.ipc.BufferCallBeforeInitHandler.BufferCallEvent;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController.CancellationCallback;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader;
 import org.apache.hadoop.hbase.security.NettyHBaseSaslRpcClientHandler;
 import org.apache.hadoop.hbase.security.SaslChallengeDecoder;
 import org.apache.hadoop.hbase.security.SaslUtil.QualityOfProtection;
@@ -247,12 +247,12 @@ class NettyRpcConnection extends RpcConnection {
   }
 
   @Override
-  public synchronized void sendRequest(final Call call, HBaseRpcController pcrc)
+  public synchronized void sendRequest(final Call call, HBaseRpcController hrc)
       throws IOException {
     if (reloginInProgress) {
       throw new IOException("Can not send request because relogin is in progress.");
     }
-    pcrc.notifyOnCancel(new RpcCallback<Object>() {
+    hrc.notifyOnCancel(new RpcCallback<Object>() {
 
       @Override
       public void run(Object parameter) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
index 5faaede..a302d48 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcDuplexHandler.java
@@ -17,9 +17,9 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import com.google.protobuf.Message;
-import com.google.protobuf.Message.Builder;
-import com.google.protobuf.TextFormat;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message.Builder;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufInputStream;
@@ -39,10 +39,10 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.CellScanner;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.codec.Codec;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.CellBlockMeta;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.CellBlockMeta;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ExceptionResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.RequestHeader;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ResponseHeader;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.ipc.RemoteException;
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
deleted file mode 100644
index 209deed..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * 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.ipc;
-
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.RpcController;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.hbase.client.ClusterConnection;
-import org.apache.hadoop.hbase.client.RegionServerCallable;
-import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
-import org.apache.hadoop.hbase.util.Bytes;
-
-/**
- * Provides clients with an RPC connection to call Coprocessor Endpoint
- * {@link com.google.protobuf.Service}s
- * against a given table region.  An instance of this class may be obtained
- * by calling {@link org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])},
- * but should normally only be used in creating a new {@link com.google.protobuf.Service} stub to
- * call the endpoint methods.
- * @see org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])
- */
-@InterfaceAudience.Private
-public class RegionCoprocessorRpcChannel extends SyncCoprocessorRpcChannel {
-  private static final Log LOG = LogFactory.getLog(RegionCoprocessorRpcChannel.class);
-
-  private final ClusterConnection connection;
-  private final TableName table;
-  private final byte[] row;
-  private byte[] lastRegion;
-  private final int operationTimeout;
-
-  private final RpcRetryingCallerFactory rpcCallerFactory;
-  private final RpcControllerFactory rpcControllerFactory;
-
-  /**
-   * Constructor
-   * @param conn connection to use
-   * @param table to connect to
-   * @param row to locate region with
-   */
-  public RegionCoprocessorRpcChannel(ClusterConnection conn, TableName table, byte[] row) {
-    this.connection = conn;
-    this.table = table;
-    this.row = row;
-    this.rpcCallerFactory = conn.getRpcRetryingCallerFactory();
-    this.rpcControllerFactory = conn.getRpcControllerFactory();
-    this.operationTimeout = conn.getConnectionConfiguration().getOperationTimeout();
-  }
-
-  @Override
-  protected Message callExecService(RpcController controller,
-      Descriptors.MethodDescriptor method, Message request, Message responsePrototype)
-          throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Call: " + method.getName() + ", " + request.toString());
-    }
-    if (row == null) {
-      throw new IllegalArgumentException("Missing row property for remote region location");
-    }
-    final ClientProtos.CoprocessorServiceCall call =
-        CoprocessorRpcUtils.buildServiceCall(row, method, request);
-    RegionServerCallable<CoprocessorServiceResponse> callable =
-        new RegionServerCallable<CoprocessorServiceResponse>(connection,
-          controller == null? this.rpcControllerFactory.newController(): controller,
-          table, row) {
-      @Override
-      protected CoprocessorServiceResponse rpcCall() throws Exception {
-        byte[] regionName = getLocation().getRegionInfo().getRegionName();
-        return ProtobufUtil.execService(getRpcController(), getStub(), call, regionName);
-      }
-    };
-    CoprocessorServiceResponse result = rpcCallerFactory.<CoprocessorServiceResponse> newCaller()
-        .callWithRetries(callable, operationTimeout);
-    Message response;
-    if (result.getValue().hasValue()) {
-      Message.Builder builder = responsePrototype.newBuilderForType();
-      ProtobufUtil.mergeFrom(builder, result.getValue().getValue());
-      response = builder.build();
-    } else {
-      response = responsePrototype.getDefaultInstanceForType();
-    }
-    lastRegion = result.getRegion().getValue().toByteArray();
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Result is region=" + Bytes.toStringBinary(lastRegion) + ", value=" + response);
-    }
-    return response;
-  }
-
-  /**
-   * Get last region this RpcChannel communicated with
-   * @return region name as byte array
-   */
-  public byte[] getLastRegion() {
-    return lastRegion;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionServerCoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionServerCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionServerCoprocessorRpcChannel.java
deleted file mode 100644
index c23d36c..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionServerCoprocessorRpcChannel.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * 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.ipc;
-
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.RpcController;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-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.ClusterConnection;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
-
-
-/**
- * Provides clients with an RPC connection to call coprocessor endpoint
- * {@link com.google.protobuf.Service}s against a given region server. An instance of this class may
- * be obtained by calling {@link org.apache.hadoop.hbase.client.HBaseAdmin#
- * coprocessorService(ServerName)}, but should normally only be used in creating a new
- * {@link com.google.protobuf.Service} stub to call the endpoint methods.
- * @see org.apache.hadoop.hbase.client.HBaseAdmin#coprocessorService(ServerName)
- */
-@InterfaceAudience.Private
-public class RegionServerCoprocessorRpcChannel extends SyncCoprocessorRpcChannel {
-  private static final Log LOG = LogFactory.getLog(RegionServerCoprocessorRpcChannel.class);
-  private final ClusterConnection connection;
-  private final ServerName serverName;
-
-  public RegionServerCoprocessorRpcChannel(ClusterConnection conn, ServerName serverName) {
-    this.connection = conn;
-    this.serverName = serverName;
-  }
-
-  @Override
-  protected Message callExecService(RpcController controller,
-      Descriptors.MethodDescriptor method, Message request, Message responsePrototype)
-          throws IOException {
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Call: " + method.getName() + ", " + request.toString());
-    }
-    final ClientProtos.CoprocessorServiceCall call =
-        CoprocessorRpcUtils.buildServiceCall(HConstants.EMPTY_BYTE_ARRAY, method, request);
-
-    // TODO: Are we retrying here? Does not seem so. We should use RetryingRpcCaller
-    CoprocessorServiceResponse result =
-        ProtobufUtil.execRegionServerService(controller, connection.getClient(serverName), call);
-    Message response;
-    if (result.getValue().hasValue()) {
-      Message.Builder builder = responsePrototype.newBuilderForType();
-      ProtobufUtil.mergeFrom(builder, result.getValue().getValue());
-      response = builder.build();
-    } else {
-      response = responsePrototype.getDefaultInstanceForType();
-    }
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Result is value=" + response);
-    }
-    return response;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
index 26a5739..c4af510 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClient.java
@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.hbase.ipc;
 
-import com.google.protobuf.BlockingRpcChannel;
-import com.google.protobuf.RpcChannel;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.BlockingRpcChannel;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcChannel;
 
 import java.io.Closeable;
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
index 5e9e97e..a60528e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcConnection.java
@@ -31,10 +31,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.codec.Codec;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
 import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ConnectionHeader;
-import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.ConnectionHeader;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos.UserInformation;
 import org.apache.hadoop.hbase.security.AuthMethod;
 import org.apache.hadoop.hbase.security.SecurityInfo;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/SyncCoprocessorRpcChannel.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/SyncCoprocessorRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/SyncCoprocessorRpcChannel.java
deleted file mode 100644
index 347d8a1..0000000
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/SyncCoprocessorRpcChannel.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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.ipc;
-
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-
-import java.io.IOException;
-
-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.ResponseConverter;
-
-/**
- * Base class which provides clients with an RPC connection to
- * call coprocessor endpoint {@link com.google.protobuf.Service}s.
- * Note that clients should not use this class directly, except through
- * {@link org.apache.hadoop.hbase.client.Table#coprocessorService(byte[])}.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public abstract class SyncCoprocessorRpcChannel implements CoprocessorRpcChannel {
-  private static final Log LOG = LogFactory.getLog(SyncCoprocessorRpcChannel.class);
-
-  @Override
-  @InterfaceAudience.Private
-  public void callMethod(Descriptors.MethodDescriptor method,
-                         RpcController controller,
-                         Message request, Message responsePrototype,
-                         RpcCallback<Message> callback) {
-    Message response = null;
-    try {
-      response = callExecService(controller, method, request, responsePrototype);
-    } catch (IOException ioe) {
-      LOG.warn("Call failed on IOException", ioe);
-      ResponseConverter.setControllerException(controller, ioe);
-    }
-    if (callback != null) {
-      callback.run(response);
-    }
-  }
-
-  @Override
-  @InterfaceAudience.Private
-  public Message callBlockingMethod(Descriptors.MethodDescriptor method,
-                                    RpcController controller,
-                                    Message request, Message responsePrototype)
-      throws ServiceException {
-    try {
-      return callExecService(controller, method, request, responsePrototype);
-    } catch (IOException ioe) {
-      throw new ServiceException("Error calling method "+method.getFullName(), ioe);
-    }
-  }
-
-  protected abstract Message callExecService(RpcController controller,
-      Descriptors.MethodDescriptor method, Message request, Message responsePrototype)
-          throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
index fd1c432..04f62d4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/master/RegionState.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
 
 /**
  * State of a Region while undergoing transitions.

http://git-wip-us.apache.org/repos/asf/hbase/blob/95c1dc93/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufMagic.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufMagic.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufMagic.java
new file mode 100644
index 0000000..f0c4b67
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufMagic.java
@@ -0,0 +1,90 @@
+/**
+ * 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.protobuf;
+
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Protobufs utility.
+ */
+@InterfaceAudience.Private
+public class ProtobufMagic {
+
+  private ProtobufMagic() {
+  }
+
+  /**
+   * Magic we put ahead of a serialized protobuf message.
+   * For example, all znode content is protobuf messages with the below magic
+   * for preamble.
+   */
+  public static final byte [] PB_MAGIC = new byte [] {'P', 'B', 'U', 'F'};
+
+  /**
+   * @param bytes Bytes to check.
+   * @return True if passed <code>bytes</code> has {@link #PB_MAGIC} for a prefix.
+   */
+  public static boolean isPBMagicPrefix(final byte [] bytes) {
+    if (bytes == null) return false;
+    return isPBMagicPrefix(bytes, 0, bytes.length);
+  }
+
+  /*
+   * Copied from Bytes.java to here
+   * hbase-common now depends on hbase-protocol
+   * Referencing Bytes.java directly would create circular dependency
+   */
+  private static int compareTo(byte[] buffer1, int offset1, int length1,
+      byte[] buffer2, int offset2, int length2) {
+    // Short circuit equal case
+    if (buffer1 == buffer2 &&
+        offset1 == offset2 &&
+        length1 == length2) {
+      return 0;
+    }
+    // Bring WritableComparator code local
+    int end1 = offset1 + length1;
+    int end2 = offset2 + length2;
+    for (int i = offset1, j = offset2; i < end1 && j < end2; i++, j++) {
+      int a = (buffer1[i] & 0xff);
+      int b = (buffer2[j] & 0xff);
+      if (a != b) {
+        return a - b;
+      }
+    }
+    return length1 - length2;
+  }
+
+  /**
+   * @param bytes Bytes to check.
+   * @param offset offset to start at
+   * @param len length to use
+   * @return True if passed <code>bytes</code> has {@link #PB_MAGIC} for a prefix.
+   */
+  public static boolean isPBMagicPrefix(final byte [] bytes, int offset, int len) {
+    if (bytes == null || len < PB_MAGIC.length) return false;
+    return compareTo(PB_MAGIC, 0, PB_MAGIC.length, bytes, offset, PB_MAGIC.length) == 0;
+  }
+
+  /**
+   * @return Length of {@link #PB_MAGIC}
+   */
+  public static int lengthOfPBMagic() {
+    return PB_MAGIC.length;
+  }
+}
\ No newline at end of file