You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2016/09/29 19:37:55 UTC

[41/51] [partial] hbase git commit: HBASE-16264 Figure how to deal with endpoints and shaded pb Shade our protobufs. Do it in a manner that makes it so we can still have in our API references to com.google.protobuf (and in REST). The c.g.p in API is for

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java
index 426b6a7..d0b6317 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/TestInterfaceAudienceAnnotations.java
@@ -76,6 +76,14 @@ public class TestInterfaceAudienceAnnotations {
     }
   }
 
+  class ShadedProtobufClassFilter implements ClassFinder.ClassFilter {
+    @Override
+    public boolean isCandidateClass(Class<?> c) {
+      return c.getPackage().getName().
+          contains("org.apache.hadoop.hbase.shaded.com.google.protobuf");
+    }
+  }
+
   /** Selects classes with one of the {@link InterfaceAudience} annotation in their class
    * declaration.
    */
@@ -273,6 +281,7 @@ public class TestInterfaceAudienceAnnotations {
       new And(new PublicClassFilter(),
               new Not(new TestClassFilter()),
               new Not(new GeneratedClassFilter()),
+              new Not(new ShadedProtobufClassFilter()),
               new Not(new IsInterfaceStabilityClassFilter()),
               new Not(new InterfaceAudienceAnnotatedClassFilter()),
               new Not(new CloverInstrumentationFilter()))
@@ -312,6 +321,7 @@ public class TestInterfaceAudienceAnnotations {
       new And(new PublicClassFilter(),
               new Not(new TestClassFilter()),
               new Not(new GeneratedClassFilter()),
+              new Not(new ShadedProtobufClassFilter()),
               new InterfaceAudiencePublicAnnotatedClassFilter(),
               new Not(new IsInterfaceStabilityClassFilter()),
               new Not(new InterfaceStabilityAnnotatedClassFilter()))
@@ -355,6 +365,7 @@ public class TestInterfaceAudienceAnnotations {
             new Not((FileNameFilter) new TestFileNameFilter()),
             new And(new PublicClassFilter(), new Not(new TestClassFilter()),
                 new Not(new GeneratedClassFilter()),
+                new Not(new ShadedProtobufClassFilter()),
                 new InterfaceAudiencePublicAnnotatedClassFilter()));
     Set<Class<?>> classes = classFinder.findClasses(false);
     return classes;

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientExponentialBackoff.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientExponentialBackoff.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientExponentialBackoff.java
index 838c40e..d7aa2f0 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientExponentialBackoff.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientExponentialBackoff.java
@@ -21,8 +21,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.client.backoff.ExponentialClientBackoffPolicy;
 import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.apache.hadoop.hbase.util.Bytes;

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
index 1ece448..df6de42 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java
@@ -49,35 +49,35 @@ import org.apache.hadoop.hbase.RegionLocations;
 import org.apache.hadoop.hbase.RegionTooBusyException;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.protobuf.generated.CellProtos;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CleanupBulkLoadRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CleanupBulkLoadResponse;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService.BlockingInterface;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetResponse;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.PrepareBulkLoadRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.PrepareBulkLoadResponse;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionResult;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ResultOrException;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.CellProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.BulkLoadHFileResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CleanupBulkLoadResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService.BlockingInterface;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.PrepareBulkLoadResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionActionResult;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ResultOrException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
 import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
-import org.apache.hadoop.hbase.util.ByteStringer;
+import org.apache.hadoop.hbase.shaded.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.Threads;
@@ -90,9 +90,9 @@ import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 
 import com.google.common.base.Stopwatch;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 
 /**
  * Test client behavior w/o setting up a cluster.

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestGet.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestGet.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestGet.java
index f370751..810f6bc 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestGet.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestGet.java
@@ -38,8 +38,8 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.security.visibility.Authorizations;
 import org.apache.hadoop.hbase.testclassification.ClientTests;

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java
index 5191880..d17dd7f 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestMetricsConnection.java
@@ -17,16 +17,16 @@
  */
 package org.apache.hadoop.hbase.client;
 
-import com.google.protobuf.ByteString;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier;
-import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.MetricsTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestProcedureFuture.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestProcedureFuture.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestProcedureFuture.java
index 641f6c4..10783e1 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestProcedureFuture.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestProcedureFuture.java
@@ -29,8 +29,8 @@ import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetProcedureResultResponse;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestScan.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestScan.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestScan.java
index 16c74df..fc1e526 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestScan.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestScan.java
@@ -26,9 +26,8 @@ import java.util.Arrays;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
-import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
 import org.apache.hadoop.hbase.security.visibility.Authorizations;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
index 80980fd..ccb5206 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestSnapshotFromAdmin.java
@@ -30,17 +30,17 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ipc.HBaseRpcController;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
-import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotRequest;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.SnapshotResponse;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.SmallTests;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 
-import com.google.protobuf.RpcController;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
 
 /**
  * Test snapshot logic from the client

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-client/src/test/java/org/apache/hadoop/hbase/exceptions/TestClientExceptionsUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/exceptions/TestClientExceptionsUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/exceptions/TestClientExceptionsUtil.java
index 968e55c..97e9574 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/exceptions/TestClientExceptionsUtil.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/exceptions/TestClientExceptionsUtil.java
@@ -18,7 +18,7 @@
  */
 package org.apache.hadoop.hbase.exceptions;
 
-import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
 import org.junit.Test;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-common/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml
index ee56970..c5f5a81 100644
--- a/hbase-common/pom.xml
+++ b/hbase-common/pom.xml
@@ -229,10 +229,6 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-protocol</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-annotations</artifactId>
       <type>test-jar</type>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
index 5bbff87..451da12 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ProcedureInfo.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
-import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
 import org.apache.hadoop.hbase.util.NonceKey;
 import org.apache.hadoop.util.StringUtils;
 
@@ -40,7 +39,7 @@ public class ProcedureInfo implements Cloneable {
   private final ProcedureState procState;
   private final long parentId;
   private final NonceKey nonceKey;
-  private final ProcedureUtil.ForeignExceptionMsg exception;
+  private final IOException exception;
   private final long lastUpdate;
   private final long startTime;
   private final byte[] result;
@@ -55,7 +54,7 @@ public class ProcedureInfo implements Cloneable {
       final ProcedureState procState,
       final long parentId,
       final NonceKey nonceKey,
-      final ProcedureUtil.ForeignExceptionMsg exception,
+      final IOException exception,
       final long lastUpdate,
       final long startTime,
       final byte[] result) {
@@ -107,7 +106,7 @@ public class ProcedureInfo implements Cloneable {
 
     if (isFailed()) {
       sb.append(", exception=\"");
-      sb.append(getExceptionMessage());
+      sb.append(this.exception.getMessage());
       sb.append("\"");
     }
     sb.append(")");
@@ -152,29 +151,15 @@ public class ProcedureInfo implements Cloneable {
 
   public IOException getException() {
     if (isFailed()) {
-      return ForeignExceptionUtil.toIOException(exception.getForeignExchangeMessage());
+      return this.exception;
     }
     return null;
   }
 
-  @InterfaceAudience.Private
-  public ProcedureUtil.ForeignExceptionMsg getForeignExceptionMessage() {
-    return exception;
-  }
-
-  public String getExceptionCause() {
-    assert isFailed();
-    return exception.getForeignExchangeMessage().getGenericException().getClassName();
-  }
-
-  public String getExceptionMessage() {
-    assert isFailed();
-    return exception.getForeignExchangeMessage().getGenericException().getMessage();
-  }
-
   public String getExceptionFullMessage() {
     assert isFailed();
-    return getExceptionCause() + " - " + getExceptionMessage();
+    final IOException e = getException();
+    return e.getCause() + " - " + e.getMessage();
   }
 
   public boolean hasResultData() {

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

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java
index 12804df..8d18db0 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/ServerName.java
@@ -18,10 +18,6 @@
  */
 package org.apache.hadoop.hbase;
 
-import com.google.common.net.HostAndPort;
-import com.google.common.net.InetAddresses;
-import com.google.protobuf.InvalidProtocolBufferException;
-
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
@@ -30,12 +26,12 @@ import java.util.regex.Pattern;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
-import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
-import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
 import org.apache.hadoop.hbase.util.Addressing;
 import org.apache.hadoop.hbase.util.Bytes;
 
+import com.google.common.net.HostAndPort;
+import com.google.common.net.InetAddresses;
+
 /**
  * Instance of an HBase ServerName.
  * A server name is used uniquely identifying a server instance in a cluster and is made
@@ -369,47 +365,4 @@ import org.apache.hadoop.hbase.util.Bytes;
     if (str == null ||str.isEmpty()) return false;
     return SERVERNAME_PATTERN.matcher(str).matches();
   }
-
-  /**
-   * Get a ServerName from the passed in data bytes.
-   * @param data Data with a serialize server name in it; can handle the old style
-   * servername where servername was host and port.  Works too with data that
-   * begins w/ the pb 'PBUF' magic and that is then followed by a protobuf that
-   * has a serialized {@link ServerName} in it.
-   * @return Returns null if <code>data</code> is null else converts passed data
-   * to a ServerName instance.
-   * @throws DeserializationException 
-   */
-  public static ServerName parseFrom(final byte [] data) throws DeserializationException {
-    if (data == null || data.length <= 0) return null;
-    if (ProtobufMagic.isPBMagicPrefix(data)) {
-      int prefixLen = ProtobufMagic.lengthOfPBMagic();
-      try {
-        ZooKeeperProtos.Master rss =
-          ZooKeeperProtos.Master.PARSER.parseFrom(data, prefixLen, data.length - prefixLen);
-        org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ServerName sn = rss.getMaster();
-        return valueOf(sn.getHostName(), sn.getPort(), sn.getStartCode());
-      } catch (InvalidProtocolBufferException e) {
-        // A failed parse of the znode is pretty catastrophic. Rather than loop
-        // retrying hoping the bad bytes will changes, and rather than change
-        // the signature on this method to add an IOE which will send ripples all
-        // over the code base, throw a RuntimeException.  This should "never" happen.
-        // Fail fast if it does.
-        throw new DeserializationException(e);
-      }
-    }
-    // The str returned could be old style -- pre hbase-1502 -- which was
-    // hostname and port seperated by a colon rather than hostname, port and
-    // startcode delimited by a ','.
-    String str = Bytes.toString(data);
-    int index = str.indexOf(ServerName.SERVERNAME_SEPARATOR);
-    if (index != -1) {
-      // Presume its ServerName serialized with versioned bytes.
-      return ServerName.parseVersionedServerName(data);
-    }
-    // Presume it a hostname:port format.
-    String hostname = Addressing.parseHostname(str);
-    int port = Addressing.parsePort(str);
-    return valueOf(hostname, port, -1L);
-  }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-common/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java
index 99f31b1..2133750 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/filter/ByteArrayComparable.java
@@ -23,9 +23,7 @@ import java.nio.ByteBuffer;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
 import org.apache.hadoop.hbase.exceptions.DeserializationException;
-import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
 import org.apache.hadoop.hbase.util.ByteBufferUtils;
-import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 
 
@@ -57,13 +55,6 @@ public abstract class ByteArrayComparable implements Comparable<byte[]> {
    */
   public abstract byte [] toByteArray();
 
-  ComparatorProtos.ByteArrayComparable convert() {
-    ComparatorProtos.ByteArrayComparable.Builder builder =
-      ComparatorProtos.ByteArrayComparable.newBuilder();
-    if (value != null) builder.setValue(ByteStringer.wrap(value));
-    return builder.build();
-  }
-
   /**
    * @param pbBytes A pb serialized {@link ByteArrayComparable} instance
    * @return An instance of {@link ByteArrayComparable} made from <code>bytes</code>

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java
index 0fc0cb5..f42615a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/io/encoding/EncodedDataBlock.java
@@ -162,6 +162,8 @@ public class EncodedDataBlock {
    * @return Size of compressed data in bytes.
    * @throws IOException
    */
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH_EXCEPTION",
+       justification="No sure what findbugs wants but looks to me like no NPE")
   public static int getCompressedSize(Algorithm algo, Compressor compressor,
       byte[] inputBuffer, int offset, int length) throws IOException {
 
@@ -186,7 +188,7 @@ public class EncodedDataBlock {
     } finally {
       nullOutputStream.close();
       compressedStream.close();
-      if (compressingStream != null) compressingStream.close();
+      compressingStream.close();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-common/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfo.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfo.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfo.java
index 0fb02d8..e68260a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfo.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfo.java
@@ -183,5 +183,4 @@ public class RSGroupInfo {
     result = 31 * result + name.hashCode();
     return result;
   }
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ForeignExceptionUtil.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ForeignExceptionUtil.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ForeignExceptionUtil.java
deleted file mode 100644
index f9aa531..0000000
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/util/ForeignExceptionUtil.java
+++ /dev/null
@@ -1,128 +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.util;
-
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.util.ArrayList;
-import java.util.List;
-
-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.protobuf.generated.ErrorHandlingProtos.GenericExceptionMessage;
-import org.apache.hadoop.hbase.protobuf.generated.ErrorHandlingProtos.StackTraceElementMessage;
-
-/**
- * Helper to convert Exceptions and StackTraces from/to protobuf.
- * (see ErrorHandling.proto for the internal of the proto messages)
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public final class ForeignExceptionUtil {
-  private ForeignExceptionUtil() { }
-
-  public static Exception toException(final ForeignExceptionMessage eem) {
-    final GenericExceptionMessage gem = eem.getGenericException();
-    final StackTraceElement[] trace = toStackTrace(gem.getTraceList());
-    try {
-      Class<?> realClass = Class.forName(gem.getClassName());
-      Class<? extends Exception> cls = realClass.asSubclass(Exception.class);
-      Constructor<? extends Exception> cn = cls.getConstructor(String.class);
-      cn.setAccessible(true);
-      Exception re = cn.newInstance(gem.getMessage());
-      re.setStackTrace(trace);
-      return re;
-    } catch (Throwable e) {
-      Exception re = new Exception(gem.getMessage());
-      re.setStackTrace(trace);
-      return re;
-    }
-  }
-
-  public static IOException toIOException(final ForeignExceptionMessage eem) {
-    GenericExceptionMessage gem = eem.getGenericException();
-    StackTraceElement[] trace = toStackTrace(gem.getTraceList());
-    RemoteException re = new RemoteException(gem.getClassName(), gem.getMessage());
-    re.setStackTrace(trace);
-    return re.unwrapRemoteException();
-  }
-
-  public static ForeignExceptionMessage toProtoForeignException(String source, Throwable t) {
-    GenericExceptionMessage.Builder gemBuilder = GenericExceptionMessage.newBuilder();
-    gemBuilder.setClassName(t.getClass().getName());
-    if (t.getMessage() != null) {
-      gemBuilder.setMessage(t.getMessage());
-    }
-    // set the stack trace, if there is one
-    List<StackTraceElementMessage> stack = toProtoStackTraceElement(t.getStackTrace());
-    if (stack != null) {
-      gemBuilder.addAllTrace(stack);
-    }
-    GenericExceptionMessage payload = gemBuilder.build();
-    ForeignExceptionMessage.Builder exception = ForeignExceptionMessage.newBuilder();
-    exception.setGenericException(payload).setSource(source);
-    return exception.build();
-  }
-
-  /**
-   * Convert a stack trace to list of {@link StackTraceElement}.
-   * @param trace the stack trace to convert to protobuf message
-   * @return <tt>null</tt> if the passed stack is <tt>null</tt>.
-   */
-  public static List<StackTraceElementMessage> toProtoStackTraceElement(StackTraceElement[] trace) {
-    // if there is no stack trace, ignore it and just return the message
-    if (trace == null) return null;
-    // build the stack trace for the message
-    List<StackTraceElementMessage> pbTrace = new ArrayList<StackTraceElementMessage>(trace.length);
-    for (StackTraceElement elem : trace) {
-      StackTraceElementMessage.Builder stackBuilder = StackTraceElementMessage.newBuilder();
-      stackBuilder.setDeclaringClass(elem.getClassName());
-      if (elem.getFileName() != null) {
-        stackBuilder.setFileName(elem.getFileName());
-      }
-      stackBuilder.setLineNumber(elem.getLineNumber());
-      stackBuilder.setMethodName(elem.getMethodName());
-      pbTrace.add(stackBuilder.build());
-    }
-    return pbTrace;
-  }
-
-  /**
-   * Unwind a serialized array of {@link StackTraceElementMessage}s to a
-   * {@link StackTraceElement}s.
-   * @param traceList list that was serialized
-   * @return the deserialized list or <tt>null</tt> if it couldn't be unwound (e.g. wasn't set on
-   *         the sender).
-   */
-  public static StackTraceElement[] toStackTrace(List<StackTraceElementMessage> traceList) {
-    if (traceList == null || traceList.size() == 0) {
-      return new StackTraceElement[0]; // empty array
-    }
-    StackTraceElement[] trace = new StackTraceElement[traceList.size()];
-    for (int i = 0; i < traceList.size(); i++) {
-      StackTraceElementMessage elem = traceList.get(i);
-      trace[i] = new StackTraceElement(
-          elem.getDeclaringClass(), elem.getMethodName(),
-          elem.hasFileName() ? elem.getFileName() : null,
-          elem.getLineNumber());
-    }
-    return trace;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-endpoint/README.txt
----------------------------------------------------------------------
diff --git a/hbase-endpoint/README.txt b/hbase-endpoint/README.txt
new file mode 100644
index 0000000..4deba00
--- /dev/null
+++ b/hbase-endpoint/README.txt
@@ -0,0 +1,42 @@
+This maven module has the protobuf definition files used by hbase Coprocessor
+Endpoints that ship with hbase core including tests. Coprocessor Endpoints
+are meant to be standalone, independent code not reliant on hbase internals.
+They define their Service using protobuf. The protobuf version they use can be
+distinct from that used by HBase internally since HBase started shading its
+protobuf references. Endpoints have no access to the shaded protobuf hbase uses.
+They do have access to the content of hbase-protocol but avoid using as much
+of this as you can as it is liable to change.
+
+The produced java classes are generated and then checked in. The reasoning is
+that they change infrequently.
+
+To regenerate the classes after making definition file changes, in here or over
+in hbase-protocol since we source some of those protos in this package, ensure
+first that the protobuf protoc tool is in your $PATH. You may need to download
+it and build it first; it is part of the protobuf package. For example, if using
+v2.5.0 of protobuf, it is obtainable from here:
+
+ https://github.com/google/protobuf/releases/tag/v2.5.0
+
+HBase uses hadoop-maven-plugins:protoc goal to invoke the protoc command. You can
+compile the protoc definitions by invoking maven with profile compile-protobuf or
+passing in compile-protobuf property.
+
+mvn compile -Dcompile-protobuf
+or
+mvn compile -Pcompile-protobuf
+
+You may also want to define protoc.path for the protoc binary
+
+mvn compile -Dcompile-protobuf -Dprotoc.path=/opt/local/bin/protoc
+
+If you have added a new proto file, you should add it to the pom.xml file first.
+Other modules also support the maven profile.
+
+After you've done the above, check it in and then check it in (or post a patch
+on a JIRA with your definition file changes and the generated files).
+
+NOTE: The maven protoc plugin is a little broken. It will only source one dir
+at a time. If changes in protobuf files, you will have to first do protoc with
+the src directory pointing back into hbase-protocol module and then rerun it
+after editing the pom to point in here to source .proto files.

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-endpoint/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-endpoint/pom.xml b/hbase-endpoint/pom.xml
new file mode 100644
index 0000000..3b0ffd7
--- /dev/null
+++ b/hbase-endpoint/pom.xml
@@ -0,0 +1,299 @@
+<?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-endpoint</artifactId>
+    <name>Apache HBase - Coprocessor Endpoint</name>
+    <description>HBase Coprocessor Endpoint implementations</description>
+
+    <properties>
+      <maven.javadoc.skip>true</maven.javadoc.skip>
+    </properties>
+
+    <build>
+      <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></ignore>
+                    </action>
+                  </pluginExecution>
+                </pluginExecutions>
+              </lifecycleMappingMetadata>
+            </configuration>
+          </plugin>
+         </plugins>
+      </pluginManagement>
+    </build>
+
+    <dependencies>
+      <!-- Intra-project dependencies -->
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-annotations</artifactId>
+        <type>test-jar</type>
+        <scope>test</scope>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-common</artifactId>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-protocol</artifactId>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-client</artifactId>
+      </dependency>
+      <!--Some of the CPEPs use hbase server-side internals; they shouldn't!
+           -->
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-server</artifactId>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-server</artifactId>
+        <type>test-jar</type>
+        <scope>test</scope>
+      </dependency>
+      <!-- General dependencies -->
+      <dependency>
+        <groupId>commons-logging</groupId>
+        <artifactId>commons-logging</artifactId>
+      </dependency>
+    </dependencies>
+
+    <profiles>
+        <!-- Skip the tests in this module -->
+        <profile>
+            <id>skip-rpc-tests</id>
+            <activation>
+                <property>
+                    <name>skip-rpc-tests</name>
+                </property>
+            </activation>
+            <properties>
+                <surefire.skipFirstPart>true</surefire.skipFirstPart>
+            </properties>
+        </profile>
+        <profile>
+          <id>compile-protobuf</id>
+          <activation>
+            <property>
+              <name>compile-protobuf</name>
+            </property>
+          </activation>
+          <build>
+            <plugins>
+              <plugin>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-maven-plugins</artifactId>
+                <executions>
+                  <execution>
+                    <id>compile-protoc</id>
+                    <phase>generate-sources</phase>
+                    <goals>
+                      <goal>protoc</goal>
+                    </goals>
+                    <configuration>
+                      <imports>
+                        <!--Reference the .protos files up in hbase-protocol so don't
+                             have to copy them local here-->
+                        <param>${basedir}/src/main/protobuf</param>
+                        <param>${basedir}/../hbase-protocol/src/main/protobuf</param>
+                      </imports>
+                      <source>
+                        <!--The last directory wins so we need to for now manually run
+                             it once with the hbase-protocol pointer and then after
+                             with pointer to the local protobuf dir-->
+                        <directory>${basedir}/../hbase-protocol/src/main/protobuf</directory>
+                        <directory>${basedir}/src/main/protobuf</directory>
+                        <!-- Unfortunately, Hadoop plugin does not support *.proto.
+                             We have to individually list every proto file here -->
+                        <includes>
+                          <!--CPEPs-->
+                          <include>Aggregate.proto</include>
+                          <include>BulkDelete.proto</include>
+                          <include>DummyRegionServerEndpoint.proto</include>
+                          <include>ColumnAggregationNullResponseProtocol.proto</include>
+                          <include>ColumnAggregationProtocol.proto</include>
+                          <include>ColumnAggregationWithErrorsProtocol.proto</include>
+                          <include>IncrementCounterProcessor.proto</include>
+                          <include>SecureBulkLoad.proto</include>
+                        </includes>
+                      </source>
+                      <!--<output>${project.build.directory}/generated-sources/java</output>-->
+                     <output>${basedir}/src/main/java/</output>
+                    </configuration>
+                  </execution>
+                </executions>
+              </plugin>
+            </plugins>
+          </build>
+        </profile>
+    <!-- Profiles for building against different hadoop versions -->
+    <!-- There are a lot of common dependencies used here, should investigate
+    if we can combine these profiles somehow -->
+
+    <!-- profile for building against Hadoop 2.x.  This is the default.  -->
+    <profile>
+      <id>hadoop-2.0</id>
+      <activation>
+        <property>
+            <!--Below formatting for dev-support/generate-hadoopX-poms.sh-->
+            <!--h2--><name>!hadoop.profile</name>
+        </property>
+      </activation>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-auth</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-client</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-hdfs</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-hdfs</artifactId>
+          <type>test-jar</type>
+          <scope>test</scope>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-minicluster</artifactId>
+          <scope>test</scope>
+        </dependency>
+        <!-- Hadoop needs Netty 3.x at test scope for the minicluster -->
+        <dependency>
+          <groupId>io.netty</groupId>
+          <artifactId>netty</artifactId>
+          <version>${netty.hadoop.version}</version>
+          <scope>test</scope>
+        </dependency>
+      </dependencies>
+    </profile>
+    <!--
+      profile for building against Hadoop 3.0.x. Activate using:
+       mvn -Dhadoop.profile=3.0
+    -->
+    <profile>
+      <id>hadoop-3.0</id>
+      <activation>
+        <property>
+          <name>hadoop.profile</name>
+          <value>3.0</value>
+        </property>
+      </activation>
+      <properties>
+        <hadoop.version>${hadoop-three.version}</hadoop.version>
+      </properties>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-minicluster</artifactId>
+        </dependency>
+        <!-- Hadoop needs Netty 3.x at test scope for the minicluster -->
+        <dependency>
+          <groupId>io.netty</groupId>
+          <artifactId>netty</artifactId>
+          <version>3.6.2.Final</version>
+          <scope>test</scope>
+        </dependency>
+      </dependencies>
+    </profile>
+    </profiles>
+</project>