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

[28/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-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/17d4b70d/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/17d4b70d/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/17d4b70d/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/17d4b70d/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/17d4b70d/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/17d4b70d/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/17d4b70d/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/17d4b70d/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/17d4b70d/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/17d4b70d/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/17d4b70d/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..387ada9
--- /dev/null
+++ b/hbase-protocol-shaded/README.txt
@@ -0,0 +1,41 @@
+These are the protobuf definition files used by core hbase. This modules
+does shading of all to do with protobuf. All of core relies on this module.
+All core references in core to protobuf are to the protobuf this module
+includes but offset by the package prefix of org.apache.hadoop.hbase.shaded.*
+as in org.apache.hadoop.hbase.shaded.protobuf.generated.* and
+org.apache.hadoop.hbase.shaded.com.google.protobuf.*.
+
+NOTE: the .protos in here are copied in an adjacent module, hbase-protocol.
+There they are non-shaded. If you make changes here, consider making them
+over in the adjacent module too. Be careful, the .proto files are not
+exactly the same; they differ in one line at least -- the location the file
+gets generated to; i.e. those in here get the extra 'shaded' in their
+package name.
+
+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, ensure first that
+the protobuf protoc tool is in your $PATH. You may need to download it and build
+it first; its 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).

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/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..46f5087
--- /dev/null
+++ b/hbase-protocol-shaded/pom.xml
@@ -0,0 +1,289 @@
+<?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>
+    </properties>
+
+    <build>
+      <plugins>
+        <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>
+                <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>
+       <plugin>
+         <!--Now unpack the shaded jar made above so the shaded classes
+             are available to subsequent modules-->
+         <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>org.apache.hbase</groupId>
+                   <artifactId>hbase-protocol-shaded</artifactId>
+                   <version>${project.version}</version>
+                   <type>jar</type>
+                   <overWrite>true</overWrite>
+                   <outputDirectory>${project.build.directory}/classes</outputDirectory>
+                 </artifactItem>
+               </artifactItems>
+             </configuration>
+           </execution>
+         </executions>
+       </plugin>
+        <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>
+      <!--BE CAREFUL! Any dependency added here needs to be
+          excluded above in the shade plugin else the dependency
+          will get bundled here-->
+      <!-- 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>2.5.0</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>
+        <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>
+                        <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>${project.build.directory}/generated-sources/java</output>-->
+                     <output>${basedir}/src/main/java/</output>
+                    </configuration>
+                  </execution>
+                </executions>
+              </plugin>
+            </plugins>
+          </build>
+        </profile>
+    </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-protocol-shaded/src/main/java/com/google/protobuf/HBaseZeroCopyByteString.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/com/google/protobuf/HBaseZeroCopyByteString.java b/hbase-protocol-shaded/src/main/java/com/google/protobuf/HBaseZeroCopyByteString.java
new file mode 100644
index 0000000..8be8ff4
--- /dev/null
+++ b/hbase-protocol-shaded/src/main/java/com/google/protobuf/HBaseZeroCopyByteString.java
@@ -0,0 +1,77 @@
+/**
+ * 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 com.google.protobuf;  // This is a lie.
+
+/**
+ * Helper class to extract byte arrays from {@link ByteString} without copy.
+ * <p>
+ * Without this protobufs would force us to copy every single byte array out
+ * of the objects de-serialized from the wire (which already do one copy, on
+ * top of the copies the JVM does to go from kernel buffer to C buffer and
+ * from C buffer to JVM buffer).
+ *
+ * @since 0.96.1
+ */
+// This will go away when we go to pb3.0.0. Meantime there are two instances of this class... one
+// in this module and another over in hbase-protocol. They are different in that one is shaded and
+// the other is not so could be problematic while the two versions exist!
+public final class HBaseZeroCopyByteString extends LiteralByteString {
+  // Gotten from AsyncHBase code base with permission.
+  /** Private constructor so this class cannot be instantiated. */
+  private HBaseZeroCopyByteString() {
+    super(null);
+    throw new UnsupportedOperationException("Should never be here.");
+  }
+
+  /**
+   * Wraps a byte array in a {@link ByteString} without copying it.
+   * @param array array to be wrapped
+   * @return wrapped array
+   */
+  public static ByteString wrap(final byte[] array) {
+    return new LiteralByteString(array);
+  }
+
+  /**
+   * Wraps a subset of a byte array in a {@link ByteString} without copying it.
+   * @param array array to be wrapped
+   * @param offset from
+   * @param length length
+   * @return wrapped array
+   */
+  public static ByteString wrap(final byte[] array, int offset, int length) {
+    return new BoundedByteString(array, offset, length);
+  }
+
+  // TODO:
+  // ZeroCopyLiteralByteString.wrap(this.buf, 0, this.count);
+
+  /**
+   * Extracts the byte array from the given {@link ByteString} without copy.
+   * @param buf A buffer from which to extract the array.  This buffer must be
+   * actually an instance of a {@code LiteralByteString}.
+   * @return byte[] representation
+   */
+  public static byte[] zeroCopyGetBytes(final ByteString buf) {
+    if (buf instanceof LiteralByteString) {
+      return ((LiteralByteString) buf).bytes;
+    }
+    throw new UnsupportedOperationException("Need a LiteralByteString, got a "
+                                            + buf.getClass().getName());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/17d4b70d/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProcedureProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProcedureProtos.java b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProcedureProtos.java
new file mode 100644
index 0000000..baf4e64
--- /dev/null
+++ b/hbase-protocol-shaded/src/main/java/org/apache/hadoop/hbase/shaded/ipc/protobuf/generated/TestProcedureProtos.java
@@ -0,0 +1,530 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: TestProcedure.proto
+
+package org.apache.hadoop.hbase.shaded.ipc.protobuf.generated;
+
+public final class TestProcedureProtos {
+  private TestProcedureProtos() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface TestTableDDLStateDataOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string table_name = 1;
+    /**
+     * <code>required string table_name = 1;</code>
+     */
+    boolean hasTableName();
+    /**
+     * <code>required string table_name = 1;</code>
+     */
+    java.lang.String getTableName();
+    /**
+     * <code>required string table_name = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getTableNameBytes();
+  }
+  /**
+   * Protobuf type {@code TestTableDDLStateData}
+   */
+  public static final class TestTableDDLStateData extends
+      com.google.protobuf.GeneratedMessage
+      implements TestTableDDLStateDataOrBuilder {
+    // Use TestTableDDLStateData.newBuilder() to construct.
+    private TestTableDDLStateData(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private TestTableDDLStateData(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final TestTableDDLStateData defaultInstance;
+    public static TestTableDDLStateData getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public TestTableDDLStateData getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private TestTableDDLStateData(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              tableName_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.internal_static_TestTableDDLStateData_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.internal_static_TestTableDDLStateData_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData.class, org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<TestTableDDLStateData> PARSER =
+        new com.google.protobuf.AbstractParser<TestTableDDLStateData>() {
+      public TestTableDDLStateData parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new TestTableDDLStateData(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<TestTableDDLStateData> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required string table_name = 1;
+    public static final int TABLE_NAME_FIELD_NUMBER = 1;
+    private java.lang.Object tableName_;
+    /**
+     * <code>required string table_name = 1;</code>
+     */
+    public boolean hasTableName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string table_name = 1;</code>
+     */
+    public java.lang.String getTableName() {
+      java.lang.Object ref = tableName_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          tableName_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string table_name = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getTableNameBytes() {
+      java.lang.Object ref = tableName_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        tableName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    private void initFields() {
+      tableName_ = "";
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasTableName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getTableNameBytes());
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getTableNameBytes());
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code TestTableDDLStateData}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateDataOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.internal_static_TestTableDDLStateData_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.internal_static_TestTableDDLStateData_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData.class, org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData.Builder.class);
+      }
+
+      // Construct using org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        tableName_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.internal_static_TestTableDDLStateData_descriptor;
+      }
+
+      public org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData getDefaultInstanceForType() {
+        return org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData.getDefaultInstance();
+      }
+
+      public org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData build() {
+        org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData buildPartial() {
+        org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData result = new org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.tableName_ = tableName_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData) {
+          return mergeFrom((org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData other) {
+        if (other == org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData.getDefaultInstance()) return this;
+        if (other.hasTableName()) {
+          bitField0_ |= 0x00000001;
+          tableName_ = other.tableName_;
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasTableName()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProcedureProtos.TestTableDDLStateData) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string table_name = 1;
+      private java.lang.Object tableName_ = "";
+      /**
+       * <code>required string table_name = 1;</code>
+       */
+      public boolean hasTableName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string table_name = 1;</code>
+       */
+      public java.lang.String getTableName() {
+        java.lang.Object ref = tableName_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          tableName_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string table_name = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getTableNameBytes() {
+        java.lang.Object ref = tableName_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          tableName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string table_name = 1;</code>
+       */
+      public Builder setTableName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        tableName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string table_name = 1;</code>
+       */
+      public Builder clearTableName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        tableName_ = getDefaultInstance().getTableName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string table_name = 1;</code>
+       */
+      public Builder setTableNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        tableName_ = value;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:TestTableDDLStateData)
+    }
+
+    static {
+      defaultInstance = new TestTableDDLStateData(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:TestTableDDLStateData)
+  }
+
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_TestTableDDLStateData_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_TestTableDDLStateData_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\023TestProcedure.proto\"+\n\025TestTableDDLSta" +
+      "teData\022\022\n\ntable_name\030\001 \002(\tBO\n5org.apache" +
+      ".hadoop.hbase.shaded.ipc.protobuf.genera" +
+      "tedB\023TestProcedureProtos\210\001\001"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_TestTableDDLStateData_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_TestTableDDLStateData_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_TestTableDDLStateData_descriptor,
+              new java.lang.String[] { "TableName", });
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+        }, assigner);
+  }
+
+  // @@protoc_insertion_point(outer_class_scope)
+}