You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by sy...@apache.org on 2016/01/27 02:39:08 UTC

[4/5] hbase git commit: HBASE-14526 Remove delayed rpc

HBASE-14526 Remove delayed rpc


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

Branch: refs/heads/hbase-12439
Commit: de62ad150d393188c8c51f3608c151324c30a731
Parents: cb53991
Author: Elliott Clark <ec...@apache.org>
Authored: Mon Jan 25 21:22:15 2016 -0800
Committer: Elliott Clark <ec...@apache.org>
Committed: Tue Jan 26 12:15:01 2016 -0800

----------------------------------------------------------------------
 hbase-server/pom.xml                            |    1 -
 .../org/apache/hadoop/hbase/ipc/CallRunner.java |   11 +-
 .../org/apache/hadoop/hbase/ipc/Delayable.java  |   73 --
 .../apache/hadoop/hbase/ipc/RpcCallContext.java |    2 +-
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  |   68 +-
 .../apache/hadoop/hbase/ipc/TestDelayedRpc.java |  367 ------
 .../generated/TestDelayedRpcProtos.java         | 1180 ------------------
 .../hbase/security/AbstractTestSecureIPC.java   |  120 +-
 .../src/test/protobuf/test_delayed_rpc.proto    |   34 -
 9 files changed, 104 insertions(+), 1752 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/de62ad15/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 26aad71..3c25094 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -858,7 +858,6 @@
                       <include>DummyRegionServerEndpoint.proto</include>
                       <include>TestProcedure.proto</include>
                       <include>test.proto</include>
-                      <include>test_delayed_rpc.proto</include>
                       <include>test_rpc_service.proto</include>
                     </includes>
                   </source>

http://git-wip-us.apache.org/repos/asf/hbase/blob/de62ad15/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
index 5b52521..64a75b9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java
@@ -125,13 +125,10 @@ public class CallRunner {
           sucessful = true;
         }
       }
-      // Set the response for undelayed calls and delayed calls with
-      // undelayed responses.
-      if (!call.isDelayed() || !call.isReturnValueDelayed()) {
-        Message param = resultPair != null ? resultPair.getFirst() : null;
-        CellScanner cells = resultPair != null ? resultPair.getSecond() : null;
-        call.setResponse(param, cells, errorThrowable, error);
-      }
+      // Set the response
+      Message param = resultPair != null ? resultPair.getFirst() : null;
+      CellScanner cells = resultPair != null ? resultPair.getSecond() : null;
+      call.setResponse(param, cells, errorThrowable, error);
       call.sendResponseIfReady();
       this.status.markComplete("Sent response");
       this.status.pause("Waiting for a call");

http://git-wip-us.apache.org/repos/asf/hbase/blob/de62ad15/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/Delayable.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/Delayable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/Delayable.java
deleted file mode 100644
index 3b61880..0000000
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/Delayable.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.ipc;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hbase.classification.InterfaceAudience;
-
-/**
- * A call whose response can be delayed by the server.
- */
-@InterfaceAudience.Private
-public interface Delayable {
-  /**
-   * Signal that the call response should be delayed, thus freeing the RPC
-   * server to handle different requests.
-   *
-   * @param delayReturnValue Controls whether the return value of the call
-   * should be set when ending the delay or right away.  There are cases when
-   * the return value can be set right away, even if the call is delayed.
-   */
-  void startDelay(boolean delayReturnValue);
-
-  /**
-   * @return is the call delayed?
-   */
-  boolean isDelayed();
-
-  /**
-   * @return is the return value delayed?
-   */
-  boolean isReturnValueDelayed();
-
-  /**
-   * Signal that the  RPC server is now allowed to send the response.
-   * @param result The value to return to the caller.  If the corresponding
-   * delay response specified that the return value should
-   * not be delayed, this parameter must be null.
-   * @throws IOException
-   */
-  void endDelay(Object result) throws IOException;
-
-  /**
-   * Signal the end of a delayed RPC, without specifying the return value.  Use
-   * this only if the return value was not delayed
-   * @throws IOException
-   */
-  void endDelay() throws IOException;
-
-  /**
-   * End the call, throwing and exception to the caller.  This works regardless
-   * of the return value being delayed.
-   * @param t Object to throw to the client.
-   * @throws IOException
-   */
-  void endDelayThrowing(Throwable t) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de62ad15/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
index f41dfbe..2bb4df2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcCallContext.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.VersionInfo;
 import org.apache.hadoop.hbase.security.User;
 
 @InterfaceAudience.Private
-public interface RpcCallContext extends Delayable {
+public interface RpcCallContext {
   /**
    * Check if the caller who made this IPC call has disconnected.
    * If called from outside the context of IPC, this does nothing.

http://git-wip-us.apache.org/repos/asf/hbase/blob/de62ad15/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 9bb6ae3..1155751 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -185,13 +185,6 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
    */
   private static final int DEFAULT_MAX_CALLQUEUE_SIZE = 1024 * 1024 * 1024;
 
-  private static final String WARN_DELAYED_CALLS = "hbase.ipc.warn.delayedrpc.number";
-
-  private static final int DEFAULT_WARN_DELAYED_CALLS = 1000;
-
-  private final int warnDelayedCalls;
-
-  private AtomicInteger delayedCalls;
   private final IPCUtil ipcUtil;
 
   private static final String AUTH_FAILED_FOR = "Auth failed for ";
@@ -305,10 +298,8 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
      * Chain of buffers to send as response.
      */
     protected BufferChain response;
-    protected boolean delayResponse;
     protected Responder responder;
-    protected boolean delayReturnValue;           // if the return value should be
-                                                  // set at call completion
+
     protected long size;                          // size of current call
     protected boolean isError;
     protected TraceInfo tinfo;
@@ -336,7 +327,6 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       this.connection = connection;
       this.timestamp = System.currentTimeMillis();
       this.response = null;
-      this.delayResponse = false;
       this.responder = responder;
       this.isError = false;
       this.size = size;
@@ -487,51 +477,6 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     }
 
     @Override
-    public synchronized void endDelay(Object result) throws IOException {
-      assert this.delayResponse;
-      assert this.delayReturnValue || result == null;
-      this.delayResponse = false;
-      delayedCalls.decrementAndGet();
-      if (this.delayReturnValue) {
-        this.setResponse(result, null, null, null);
-      }
-      this.responder.doRespond(this);
-    }
-
-    @Override
-    public synchronized void endDelay() throws IOException {
-      this.endDelay(null);
-    }
-
-    @Override
-    public synchronized void startDelay(boolean delayReturnValue) {
-      assert !this.delayResponse;
-      this.delayResponse = true;
-      this.delayReturnValue = delayReturnValue;
-      int numDelayed = delayedCalls.incrementAndGet();
-      if (numDelayed > warnDelayedCalls) {
-        LOG.warn("Too many delayed calls: limit " + warnDelayedCalls + " current " + numDelayed);
-      }
-    }
-
-    @Override
-    public synchronized void endDelayThrowing(Throwable t) throws IOException {
-      this.setResponse(null, null, t, StringUtils.stringifyException(t));
-      this.delayResponse = false;
-      this.sendResponseIfReady();
-    }
-
-    @Override
-    public synchronized boolean isDelayed() {
-      return this.delayResponse;
-    }
-
-    @Override
-    public synchronized boolean isReturnValueDelayed() {
-      return this.delayReturnValue;
-    }
-
-    @Override
     public boolean isClientCellBlockSupported() {
       return this.connection != null && this.connection.codec != null;
     }
@@ -567,15 +512,8 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
       responseBlockSize += blockSize;
     }
 
-    /**
-     * If we have a response, and delay is not set, then respond
-     * immediately.  Otherwise, do not respond to client.  This is
-     * called by the RPC code in the context of the Handler thread.
-     */
     public synchronized void sendResponseIfReady() throws IOException {
-      if (!this.delayResponse) {
-        this.responder.doRespond(this);
-      }
+      this.responder.doRespond(this);
     }
 
     public UserGroupInformation getRemoteUser() {
@@ -2082,8 +2020,6 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     this.tcpNoDelay = conf.getBoolean("hbase.ipc.server.tcpnodelay", true);
     this.tcpKeepAlive = conf.getBoolean("hbase.ipc.server.tcpkeepalive", true);
 
-    this.warnDelayedCalls = conf.getInt(WARN_DELAYED_CALLS, DEFAULT_WARN_DELAYED_CALLS);
-    this.delayedCalls = new AtomicInteger(0);
     this.ipcUtil = new IPCUtil(conf);
 
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/de62ad15/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java
deleted file mode 100644
index d379722..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestDelayedRpc.java
+++ /dev/null
@@ -1,367 +0,0 @@
-/**
- *
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hbase.ipc;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseConfiguration;
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.testclassification.MediumTests;
-import org.apache.hadoop.hbase.testclassification.RPCTests;
-import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos;
-import org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg;
-import org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse;
-import org.apache.hadoop.hbase.security.User;
-import org.apache.log4j.AppenderSkeleton;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.apache.log4j.spi.LoggingEvent;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.google.common.collect.Lists;
-import com.google.protobuf.BlockingRpcChannel;
-import com.google.protobuf.BlockingService;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-
-/**
- * Test that delayed RPCs work. Fire up three calls, the first of which should
- * be delayed. Check that the last two, which are undelayed, return before the
- * first one.
- */
-@Category({RPCTests.class, MediumTests.class}) // Fails sometimes with small tests
-public class TestDelayedRpc {
-  private static final Log LOG = LogFactory.getLog(TestDelayedRpc.class);
-  public static RpcServerInterface rpcServer;
-  public static final int UNDELAYED = 0;
-  public static final int DELAYED = 1;
-  private static final int RPC_CLIENT_TIMEOUT = 30000;
-
-  @Test (timeout=60000)
-  public void testDelayedRpcImmediateReturnValue() throws Exception {
-    testDelayedRpc(false);
-  }
-
-  @Test (timeout=60000)
-  public void testDelayedRpcDelayedReturnValue() throws Exception {
-    testDelayedRpc(true);
-  }
-
-  private void testDelayedRpc(boolean delayReturnValue) throws Exception {
-    LOG.info("Running testDelayedRpc delayReturnValue=" + delayReturnValue);
-    Configuration conf = HBaseConfiguration.create();
-    InetSocketAddress isa = new InetSocketAddress("localhost", 0);
-    TestDelayedImplementation instance = new TestDelayedImplementation(delayReturnValue);
-    BlockingService service =
-      TestDelayedRpcProtos.TestDelayedService.newReflectiveBlockingService(instance);
-    rpcServer = new RpcServer(null, "testDelayedRpc",
-        Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(service, null)),
-        isa,
-        conf,
-        new FifoRpcScheduler(conf, 1));
-    rpcServer.start();
-    RpcClient rpcClient = RpcClientFactory.createClient(
-        conf, HConstants.DEFAULT_CLUSTER_ID.toString());
-    try {
-      InetSocketAddress address = rpcServer.getListenerAddress();
-      if (address == null) {
-        throw new IOException("Listener channel is closed");
-      }
-      BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(
-          ServerName.valueOf(address.getHostName(),address.getPort(), System.currentTimeMillis()),
-          User.getCurrent(), RPC_CLIENT_TIMEOUT);
-      TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub =
-        TestDelayedRpcProtos.TestDelayedService.newBlockingStub(channel);
-      List<Integer> results = new ArrayList<Integer>();
-      // Setting true sets 'delayed' on the client.
-      TestThread th1 = new TestThread(stub, true, results);
-      // Setting 'false' means we will return UNDELAYED as response immediately.
-      TestThread th2 = new TestThread(stub, false, results);
-      TestThread th3 = new TestThread(stub, false, results);
-      th1.start();
-      Thread.sleep(100);
-      th2.start();
-      Thread.sleep(200);
-      th3.start();
-
-      th1.join();
-      th2.join();
-      th3.join();
-
-      // We should get the two undelayed responses first.
-      assertEquals(UNDELAYED, results.get(0).intValue());
-      assertEquals(UNDELAYED, results.get(1).intValue());
-      assertEquals(results.get(2).intValue(), delayReturnValue ? DELAYED :  0xDEADBEEF);
-    } finally {
-      rpcClient.close();
-    }
-  }
-
-  private static class ListAppender extends AppenderSkeleton {
-    private final List<String> messages = new ArrayList<String>();
-
-    @Override
-    protected void append(LoggingEvent event) {
-      messages.add(event.getMessage().toString());
-    }
-
-    @Override
-    public void close() {
-    }
-
-    @Override
-    public boolean requiresLayout() {
-      return false;
-    }
-
-    public List<String> getMessages() {
-      return messages;
-    }
-  }
-
-  /**
-   * Tests that we see a WARN message in the logs.
-   * @throws Exception
-   */
-  @Test (timeout=60000)
-  public void testTooManyDelayedRpcs() throws Exception {
-    Configuration conf = HBaseConfiguration.create();
-    final int MAX_DELAYED_RPC = 10;
-    conf.setInt("hbase.ipc.warn.delayedrpc.number", MAX_DELAYED_RPC);
-    // Set up an appender to catch the "Too many delayed calls" that we expect.
-    ListAppender listAppender = new ListAppender();
-    Logger log = Logger.getLogger(RpcServer.class);
-    log.addAppender(listAppender);
-    log.setLevel(Level.WARN);
-
-
-    InetSocketAddress isa = new InetSocketAddress("localhost", 0);
-    TestDelayedImplementation instance = new TestDelayedImplementation(true);
-    BlockingService service =
-      TestDelayedRpcProtos.TestDelayedService.newReflectiveBlockingService(instance);
-    rpcServer = new RpcServer(null, "testTooManyDelayedRpcs",
-      Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(service, null)),
-        isa,
-        conf,
-        new FifoRpcScheduler(conf, 1));
-    rpcServer.start();
-    RpcClient rpcClient = RpcClientFactory.createClient(
-        conf, HConstants.DEFAULT_CLUSTER_ID.toString());
-    try {
-      InetSocketAddress address = rpcServer.getListenerAddress();
-      if (address == null) {
-        throw new IOException("Listener channel is closed");
-      }
-      BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(
-          ServerName.valueOf(address.getHostName(),address.getPort(), System.currentTimeMillis()),
-          User.getCurrent(), RPC_CLIENT_TIMEOUT);
-      TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub =
-        TestDelayedRpcProtos.TestDelayedService.newBlockingStub(channel);
-      Thread threads[] = new Thread[MAX_DELAYED_RPC + 1];
-      for (int i = 0; i < MAX_DELAYED_RPC; i++) {
-        threads[i] = new TestThread(stub, true, null);
-        threads[i].start();
-      }
-
-      /* No warnings till here. */
-      assertTrue(listAppender.getMessages().isEmpty());
-
-      /* This should give a warning. */
-      threads[MAX_DELAYED_RPC] = new TestThread(stub, true, null);
-      threads[MAX_DELAYED_RPC].start();
-
-      for (int i = 0; i < MAX_DELAYED_RPC; i++) {
-        threads[i].join();
-      }
-
-      assertFalse(listAppender.getMessages().isEmpty());
-      assertTrue(listAppender.getMessages().get(0).startsWith("Too many delayed calls"));
-
-      log.removeAppender(listAppender);
-    } finally {
-      rpcClient.close();
-    }
-  }
-
-  public static class TestDelayedImplementation
-  implements TestDelayedRpcProtos.TestDelayedService.BlockingInterface {
-    /**
-     * Should the return value of delayed call be set at the end of the delay
-     * or at call return.
-     */
-    private final boolean delayReturnValue;
-
-    /**
-     * @param delayReturnValue Should the response to the delayed call be set
-     * at the start or the end of the delay.
-     */
-    public TestDelayedImplementation(boolean delayReturnValue) {
-      this.delayReturnValue = delayReturnValue;
-    }
-
-    @Override
-    public TestResponse test(final RpcController rpcController, final TestArg testArg)
-    throws ServiceException {
-      boolean delay = testArg.getDelay();
-      TestResponse.Builder responseBuilder = TestResponse.newBuilder();
-      if (!delay) {
-        responseBuilder.setResponse(UNDELAYED);
-        return responseBuilder.build();
-      }
-      final Delayable call = RpcServer.getCurrentCall();
-      call.startDelay(delayReturnValue);
-      new Thread() {
-        @Override
-        public void run() {
-          try {
-            Thread.sleep(500);
-            TestResponse.Builder responseBuilder = TestResponse.newBuilder();
-            call.endDelay(delayReturnValue ?
-                responseBuilder.setResponse(DELAYED).build() : null);
-          } catch (Exception e) {
-            e.printStackTrace();
-          }
-        }
-      }.start();
-      // This value should go back to client only if the response is set
-      // immediately at delay time.
-      responseBuilder.setResponse(0xDEADBEEF);
-      return responseBuilder.build();
-    }
-  }
-
-  public static class TestThread extends Thread {
-    private final TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub;
-    private final boolean delay;
-    private final List<Integer> results;
-
-    public TestThread(TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub,
-        boolean delay, List<Integer> results) {
-      this.stub = stub;
-      this.delay = delay;
-      this.results = results;
-    }
-
-    @Override
-    public void run() {
-      Integer result;
-      try {
-        result = new Integer(stub.test(null, TestArg.newBuilder().setDelay(delay).build()).
-          getResponse());
-      } catch (ServiceException e) {
-        throw new RuntimeException(e);
-      }
-      if (results != null) {
-        synchronized (results) {
-          results.add(result);
-        }
-      }
-    }
-  }
-
-  @Test
-  public void testEndDelayThrowing() throws IOException {
-    Configuration conf = HBaseConfiguration.create();
-    InetSocketAddress isa = new InetSocketAddress("localhost", 0);
-    FaultyTestDelayedImplementation instance = new FaultyTestDelayedImplementation();
-    BlockingService service =
-      TestDelayedRpcProtos.TestDelayedService.newReflectiveBlockingService(instance);
-    rpcServer = new RpcServer(null, "testEndDelayThrowing",
-        Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(service, null)),
-        isa,
-        conf,
-        new FifoRpcScheduler(conf, 1));
-    rpcServer.start();
-    RpcClient rpcClient = RpcClientFactory.createClient(
-        conf, HConstants.DEFAULT_CLUSTER_ID.toString());
-    try {
-      InetSocketAddress address = rpcServer.getListenerAddress();
-      if (address == null) {
-        throw new IOException("Listener channel is closed");
-      }
-      BlockingRpcChannel channel = rpcClient.createBlockingRpcChannel(
-          ServerName.valueOf(address.getHostName(),address.getPort(), System.currentTimeMillis()),
-        User.getCurrent(), 1000);
-      TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub =
-        TestDelayedRpcProtos.TestDelayedService.newBlockingStub(channel);
-
-      int result = 0xDEADBEEF;
-
-      try {
-        result = stub.test(null, TestArg.newBuilder().setDelay(false).build()).getResponse();
-      } catch (Exception e) {
-        fail("No exception should have been thrown.");
-      }
-      assertEquals(result, UNDELAYED);
-
-      boolean caughtException = false;
-      try {
-        result = stub.test(null, TestArg.newBuilder().setDelay(true).build()).getResponse();
-      } catch(Exception e) {
-        // Exception thrown by server is enclosed in a RemoteException.
-        if (e.getCause().getMessage().contains("java.lang.Exception: Something went wrong")) {
-          caughtException = true;
-        }
-        LOG.warn("Caught exception, expected=" + caughtException);
-      }
-      assertTrue(caughtException);
-    } finally {
-      rpcClient.close();
-    }
-  }
-
-  /**
-   * Delayed calls to this class throw an exception.
-   */
-  private static class FaultyTestDelayedImplementation extends TestDelayedImplementation {
-    public FaultyTestDelayedImplementation() {
-      super(false);
-    }
-
-    @Override
-    public TestResponse test(RpcController rpcController, TestArg arg)
-    throws ServiceException {
-      LOG.info("In faulty test, delay=" + arg.getDelay());
-      if (!arg.getDelay()) return TestResponse.newBuilder().setResponse(UNDELAYED).build();
-      Delayable call = RpcServer.getCurrentCall();
-      call.startDelay(true);
-      LOG.info("In faulty test, delaying");
-      try {
-        call.endDelayThrowing(new Exception("Something went wrong"));
-      } catch (IOException e) {
-        e.printStackTrace();
-      }
-      // Client will receive the Exception, not this value.
-      return TestResponse.newBuilder().setResponse(DELAYED).build();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de62ad15/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestDelayedRpcProtos.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestDelayedRpcProtos.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestDelayedRpcProtos.java
deleted file mode 100644
index 996ee86..0000000
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/protobuf/generated/TestDelayedRpcProtos.java
+++ /dev/null
@@ -1,1180 +0,0 @@
-// Generated by the protocol buffer compiler.  DO NOT EDIT!
-// source: test_delayed_rpc.proto
-
-package org.apache.hadoop.hbase.ipc.protobuf.generated;
-
-public final class TestDelayedRpcProtos {
-  private TestDelayedRpcProtos() {}
-  public static void registerAllExtensions(
-      com.google.protobuf.ExtensionRegistry registry) {
-  }
-  public interface TestArgOrBuilder
-      extends com.google.protobuf.MessageOrBuilder {
-
-    // required bool delay = 1;
-    /**
-     * <code>required bool delay = 1;</code>
-     */
-    boolean hasDelay();
-    /**
-     * <code>required bool delay = 1;</code>
-     */
-    boolean getDelay();
-  }
-  /**
-   * Protobuf type {@code TestArg}
-   */
-  public static final class TestArg extends
-      com.google.protobuf.GeneratedMessage
-      implements TestArgOrBuilder {
-    // Use TestArg.newBuilder() to construct.
-    private TestArg(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private TestArg(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final TestArg defaultInstance;
-    public static TestArg getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    public TestArg getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private TestArg(
-        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 8: {
-              bitField0_ |= 0x00000001;
-              delay_ = input.readBool();
-              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.ipc.protobuf.generated.TestDelayedRpcProtos.internal_static_TestArg_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.internal_static_TestArg_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg.class, org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<TestArg> PARSER =
-        new com.google.protobuf.AbstractParser<TestArg>() {
-      public TestArg parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new TestArg(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<TestArg> getParserForType() {
-      return PARSER;
-    }
-
-    private int bitField0_;
-    // required bool delay = 1;
-    public static final int DELAY_FIELD_NUMBER = 1;
-    private boolean delay_;
-    /**
-     * <code>required bool delay = 1;</code>
-     */
-    public boolean hasDelay() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>required bool delay = 1;</code>
-     */
-    public boolean getDelay() {
-      return delay_;
-    }
-
-    private void initFields() {
-      delay_ = false;
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      if (!hasDelay()) {
-        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.writeBool(1, delay_);
-      }
-      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
-          .computeBoolSize(1, delay_);
-      }
-      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();
-    }
-
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg)) {
-        return super.equals(obj);
-      }
-      org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg other = (org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg) obj;
-
-      boolean result = true;
-      result = result && (hasDelay() == other.hasDelay());
-      if (hasDelay()) {
-        result = result && (getDelay()
-            == other.getDelay());
-      }
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
-
-    private int memoizedHashCode = 0;
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasDelay()) {
-        hash = (37 * hash) + DELAY_FIELD_NUMBER;
-        hash = (53 * hash) + hashBoolean(getDelay());
-      }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg 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.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg 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.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
-    }
-    public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg 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.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg 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.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg 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 TestArg}
-     */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArgOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.internal_static_TestArg_descriptor;
-      }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.internal_static_TestArg_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg.class, org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg.Builder.class);
-      }
-
-      // Construct using org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg.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();
-        delay_ = false;
-        bitField0_ = (bitField0_ & ~0x00000001);
-        return this;
-      }
-
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
-
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.internal_static_TestArg_descriptor;
-      }
-
-      public org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg.getDefaultInstance();
-      }
-
-      public org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg build() {
-        org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg buildPartial() {
-        org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg result = new org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.delay_ = delay_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg) {
-          return mergeFrom((org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg other) {
-        if (other == org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg.getDefaultInstance()) return this;
-        if (other.hasDelay()) {
-          setDelay(other.getDelay());
-        }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        if (!hasDelay()) {
-          
-          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.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg) e.getUnfinishedMessage();
-          throw e;
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      // required bool delay = 1;
-      private boolean delay_ ;
-      /**
-       * <code>required bool delay = 1;</code>
-       */
-      public boolean hasDelay() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>required bool delay = 1;</code>
-       */
-      public boolean getDelay() {
-        return delay_;
-      }
-      /**
-       * <code>required bool delay = 1;</code>
-       */
-      public Builder setDelay(boolean value) {
-        bitField0_ |= 0x00000001;
-        delay_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required bool delay = 1;</code>
-       */
-      public Builder clearDelay() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        delay_ = false;
-        onChanged();
-        return this;
-      }
-
-      // @@protoc_insertion_point(builder_scope:TestArg)
-    }
-
-    static {
-      defaultInstance = new TestArg(true);
-      defaultInstance.initFields();
-    }
-
-    // @@protoc_insertion_point(class_scope:TestArg)
-  }
-
-  public interface TestResponseOrBuilder
-      extends com.google.protobuf.MessageOrBuilder {
-
-    // required int32 response = 1;
-    /**
-     * <code>required int32 response = 1;</code>
-     */
-    boolean hasResponse();
-    /**
-     * <code>required int32 response = 1;</code>
-     */
-    int getResponse();
-  }
-  /**
-   * Protobuf type {@code TestResponse}
-   */
-  public static final class TestResponse extends
-      com.google.protobuf.GeneratedMessage
-      implements TestResponseOrBuilder {
-    // Use TestResponse.newBuilder() to construct.
-    private TestResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
-      super(builder);
-      this.unknownFields = builder.getUnknownFields();
-    }
-    private TestResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
-
-    private static final TestResponse defaultInstance;
-    public static TestResponse getDefaultInstance() {
-      return defaultInstance;
-    }
-
-    public TestResponse getDefaultInstanceForType() {
-      return defaultInstance;
-    }
-
-    private final com.google.protobuf.UnknownFieldSet unknownFields;
-    @java.lang.Override
-    public final com.google.protobuf.UnknownFieldSet
-        getUnknownFields() {
-      return this.unknownFields;
-    }
-    private TestResponse(
-        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 8: {
-              bitField0_ |= 0x00000001;
-              response_ = input.readInt32();
-              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.ipc.protobuf.generated.TestDelayedRpcProtos.internal_static_TestResponse_descriptor;
-    }
-
-    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-        internalGetFieldAccessorTable() {
-      return org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.internal_static_TestResponse_fieldAccessorTable
-          .ensureFieldAccessorsInitialized(
-              org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse.class, org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse.Builder.class);
-    }
-
-    public static com.google.protobuf.Parser<TestResponse> PARSER =
-        new com.google.protobuf.AbstractParser<TestResponse>() {
-      public TestResponse parsePartialFrom(
-          com.google.protobuf.CodedInputStream input,
-          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-          throws com.google.protobuf.InvalidProtocolBufferException {
-        return new TestResponse(input, extensionRegistry);
-      }
-    };
-
-    @java.lang.Override
-    public com.google.protobuf.Parser<TestResponse> getParserForType() {
-      return PARSER;
-    }
-
-    private int bitField0_;
-    // required int32 response = 1;
-    public static final int RESPONSE_FIELD_NUMBER = 1;
-    private int response_;
-    /**
-     * <code>required int32 response = 1;</code>
-     */
-    public boolean hasResponse() {
-      return ((bitField0_ & 0x00000001) == 0x00000001);
-    }
-    /**
-     * <code>required int32 response = 1;</code>
-     */
-    public int getResponse() {
-      return response_;
-    }
-
-    private void initFields() {
-      response_ = 0;
-    }
-    private byte memoizedIsInitialized = -1;
-    public final boolean isInitialized() {
-      byte isInitialized = memoizedIsInitialized;
-      if (isInitialized != -1) return isInitialized == 1;
-
-      if (!hasResponse()) {
-        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.writeInt32(1, response_);
-      }
-      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
-          .computeInt32Size(1, response_);
-      }
-      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();
-    }
-
-    @java.lang.Override
-    public boolean equals(final java.lang.Object obj) {
-      if (obj == this) {
-       return true;
-      }
-      if (!(obj instanceof org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse)) {
-        return super.equals(obj);
-      }
-      org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse other = (org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse) obj;
-
-      boolean result = true;
-      result = result && (hasResponse() == other.hasResponse());
-      if (hasResponse()) {
-        result = result && (getResponse()
-            == other.getResponse());
-      }
-      result = result &&
-          getUnknownFields().equals(other.getUnknownFields());
-      return result;
-    }
-
-    private int memoizedHashCode = 0;
-    @java.lang.Override
-    public int hashCode() {
-      if (memoizedHashCode != 0) {
-        return memoizedHashCode;
-      }
-      int hash = 41;
-      hash = (19 * hash) + getDescriptorForType().hashCode();
-      if (hasResponse()) {
-        hash = (37 * hash) + RESPONSE_FIELD_NUMBER;
-        hash = (53 * hash) + getResponse();
-      }
-      hash = (29 * hash) + getUnknownFields().hashCode();
-      memoizedHashCode = hash;
-      return hash;
-    }
-
-    public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse parseFrom(
-        com.google.protobuf.ByteString data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse 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.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse parseFrom(byte[] data)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data);
-    }
-    public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse parseFrom(
-        byte[] data,
-        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
-        throws com.google.protobuf.InvalidProtocolBufferException {
-      return PARSER.parseFrom(data, extensionRegistry);
-    }
-    public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse parseFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse 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.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse parseDelimitedFrom(java.io.InputStream input)
-        throws java.io.IOException {
-      return PARSER.parseDelimitedFrom(input);
-    }
-    public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse 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.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse parseFrom(
-        com.google.protobuf.CodedInputStream input)
-        throws java.io.IOException {
-      return PARSER.parseFrom(input);
-    }
-    public static org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse 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.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse 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 TestResponse}
-     */
-    public static final class Builder extends
-        com.google.protobuf.GeneratedMessage.Builder<Builder>
-       implements org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponseOrBuilder {
-      public static final com.google.protobuf.Descriptors.Descriptor
-          getDescriptor() {
-        return org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.internal_static_TestResponse_descriptor;
-      }
-
-      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
-          internalGetFieldAccessorTable() {
-        return org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.internal_static_TestResponse_fieldAccessorTable
-            .ensureFieldAccessorsInitialized(
-                org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse.class, org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse.Builder.class);
-      }
-
-      // Construct using org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse.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();
-        response_ = 0;
-        bitField0_ = (bitField0_ & ~0x00000001);
-        return this;
-      }
-
-      public Builder clone() {
-        return create().mergeFrom(buildPartial());
-      }
-
-      public com.google.protobuf.Descriptors.Descriptor
-          getDescriptorForType() {
-        return org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.internal_static_TestResponse_descriptor;
-      }
-
-      public org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse getDefaultInstanceForType() {
-        return org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse.getDefaultInstance();
-      }
-
-      public org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse build() {
-        org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse result = buildPartial();
-        if (!result.isInitialized()) {
-          throw newUninitializedMessageException(result);
-        }
-        return result;
-      }
-
-      public org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse buildPartial() {
-        org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse result = new org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse(this);
-        int from_bitField0_ = bitField0_;
-        int to_bitField0_ = 0;
-        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
-          to_bitField0_ |= 0x00000001;
-        }
-        result.response_ = response_;
-        result.bitField0_ = to_bitField0_;
-        onBuilt();
-        return result;
-      }
-
-      public Builder mergeFrom(com.google.protobuf.Message other) {
-        if (other instanceof org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse) {
-          return mergeFrom((org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse)other);
-        } else {
-          super.mergeFrom(other);
-          return this;
-        }
-      }
-
-      public Builder mergeFrom(org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse other) {
-        if (other == org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse.getDefaultInstance()) return this;
-        if (other.hasResponse()) {
-          setResponse(other.getResponse());
-        }
-        this.mergeUnknownFields(other.getUnknownFields());
-        return this;
-      }
-
-      public final boolean isInitialized() {
-        if (!hasResponse()) {
-          
-          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.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse parsedMessage = null;
-        try {
-          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
-        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
-          parsedMessage = (org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse) e.getUnfinishedMessage();
-          throw e;
-        } finally {
-          if (parsedMessage != null) {
-            mergeFrom(parsedMessage);
-          }
-        }
-        return this;
-      }
-      private int bitField0_;
-
-      // required int32 response = 1;
-      private int response_ ;
-      /**
-       * <code>required int32 response = 1;</code>
-       */
-      public boolean hasResponse() {
-        return ((bitField0_ & 0x00000001) == 0x00000001);
-      }
-      /**
-       * <code>required int32 response = 1;</code>
-       */
-      public int getResponse() {
-        return response_;
-      }
-      /**
-       * <code>required int32 response = 1;</code>
-       */
-      public Builder setResponse(int value) {
-        bitField0_ |= 0x00000001;
-        response_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>required int32 response = 1;</code>
-       */
-      public Builder clearResponse() {
-        bitField0_ = (bitField0_ & ~0x00000001);
-        response_ = 0;
-        onChanged();
-        return this;
-      }
-
-      // @@protoc_insertion_point(builder_scope:TestResponse)
-    }
-
-    static {
-      defaultInstance = new TestResponse(true);
-      defaultInstance.initFields();
-    }
-
-    // @@protoc_insertion_point(class_scope:TestResponse)
-  }
-
-  /**
-   * Protobuf service {@code TestDelayedService}
-   */
-  public static abstract class TestDelayedService
-      implements com.google.protobuf.Service {
-    protected TestDelayedService() {}
-
-    public interface Interface {
-      /**
-       * <code>rpc test(.TestArg) returns (.TestResponse);</code>
-       */
-      public abstract void test(
-          com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg request,
-          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse> done);
-
-    }
-
-    public static com.google.protobuf.Service newReflectiveService(
-        final Interface impl) {
-      return new TestDelayedService() {
-        @java.lang.Override
-        public  void test(
-            com.google.protobuf.RpcController controller,
-            org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg request,
-            com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse> done) {
-          impl.test(controller, request, done);
-        }
-
-      };
-    }
-
-    public static com.google.protobuf.BlockingService
-        newReflectiveBlockingService(final BlockingInterface impl) {
-      return new com.google.protobuf.BlockingService() {
-        public final com.google.protobuf.Descriptors.ServiceDescriptor
-            getDescriptorForType() {
-          return getDescriptor();
-        }
-
-        public final com.google.protobuf.Message callBlockingMethod(
-            com.google.protobuf.Descriptors.MethodDescriptor method,
-            com.google.protobuf.RpcController controller,
-            com.google.protobuf.Message request)
-            throws com.google.protobuf.ServiceException {
-          if (method.getService() != getDescriptor()) {
-            throw new java.lang.IllegalArgumentException(
-              "Service.callBlockingMethod() given method descriptor for " +
-              "wrong service type.");
-          }
-          switch(method.getIndex()) {
-            case 0:
-              return impl.test(controller, (org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg)request);
-            default:
-              throw new java.lang.AssertionError("Can't get here.");
-          }
-        }
-
-        public final com.google.protobuf.Message
-            getRequestPrototype(
-            com.google.protobuf.Descriptors.MethodDescriptor method) {
-          if (method.getService() != getDescriptor()) {
-            throw new java.lang.IllegalArgumentException(
-              "Service.getRequestPrototype() given method " +
-              "descriptor for wrong service type.");
-          }
-          switch(method.getIndex()) {
-            case 0:
-              return org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg.getDefaultInstance();
-            default:
-              throw new java.lang.AssertionError("Can't get here.");
-          }
-        }
-
-        public final com.google.protobuf.Message
-            getResponsePrototype(
-            com.google.protobuf.Descriptors.MethodDescriptor method) {
-          if (method.getService() != getDescriptor()) {
-            throw new java.lang.IllegalArgumentException(
-              "Service.getResponsePrototype() given method " +
-              "descriptor for wrong service type.");
-          }
-          switch(method.getIndex()) {
-            case 0:
-              return org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse.getDefaultInstance();
-            default:
-              throw new java.lang.AssertionError("Can't get here.");
-          }
-        }
-
-      };
-    }
-
-    /**
-     * <code>rpc test(.TestArg) returns (.TestResponse);</code>
-     */
-    public abstract void test(
-        com.google.protobuf.RpcController controller,
-        org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg request,
-        com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse> done);
-
-    public static final
-        com.google.protobuf.Descriptors.ServiceDescriptor
-        getDescriptor() {
-      return org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.getDescriptor().getServices().get(0);
-    }
-    public final com.google.protobuf.Descriptors.ServiceDescriptor
-        getDescriptorForType() {
-      return getDescriptor();
-    }
-
-    public final void callMethod(
-        com.google.protobuf.Descriptors.MethodDescriptor method,
-        com.google.protobuf.RpcController controller,
-        com.google.protobuf.Message request,
-        com.google.protobuf.RpcCallback<
-          com.google.protobuf.Message> done) {
-      if (method.getService() != getDescriptor()) {
-        throw new java.lang.IllegalArgumentException(
-          "Service.callMethod() given method descriptor for wrong " +
-          "service type.");
-      }
-      switch(method.getIndex()) {
-        case 0:
-          this.test(controller, (org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg)request,
-            com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse>specializeCallback(
-              done));
-          return;
-        default:
-          throw new java.lang.AssertionError("Can't get here.");
-      }
-    }
-
-    public final com.google.protobuf.Message
-        getRequestPrototype(
-        com.google.protobuf.Descriptors.MethodDescriptor method) {
-      if (method.getService() != getDescriptor()) {
-        throw new java.lang.IllegalArgumentException(
-          "Service.getRequestPrototype() given method " +
-          "descriptor for wrong service type.");
-      }
-      switch(method.getIndex()) {
-        case 0:
-          return org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg.getDefaultInstance();
-        default:
-          throw new java.lang.AssertionError("Can't get here.");
-      }
-    }
-
-    public final com.google.protobuf.Message
-        getResponsePrototype(
-        com.google.protobuf.Descriptors.MethodDescriptor method) {
-      if (method.getService() != getDescriptor()) {
-        throw new java.lang.IllegalArgumentException(
-          "Service.getResponsePrototype() given method " +
-          "descriptor for wrong service type.");
-      }
-      switch(method.getIndex()) {
-        case 0:
-          return org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse.getDefaultInstance();
-        default:
-          throw new java.lang.AssertionError("Can't get here.");
-      }
-    }
-
-    public static Stub newStub(
-        com.google.protobuf.RpcChannel channel) {
-      return new Stub(channel);
-    }
-
-    public static final class Stub extends org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestDelayedService implements Interface {
-      private Stub(com.google.protobuf.RpcChannel channel) {
-        this.channel = channel;
-      }
-
-      private final com.google.protobuf.RpcChannel channel;
-
-      public com.google.protobuf.RpcChannel getChannel() {
-        return channel;
-      }
-
-      public  void test(
-          com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg request,
-          com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse> done) {
-        channel.callMethod(
-          getDescriptor().getMethods().get(0),
-          controller,
-          request,
-          org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse.getDefaultInstance(),
-          com.google.protobuf.RpcUtil.generalizeCallback(
-            done,
-            org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse.class,
-            org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse.getDefaultInstance()));
-      }
-    }
-
-    public static BlockingInterface newBlockingStub(
-        com.google.protobuf.BlockingRpcChannel channel) {
-      return new BlockingStub(channel);
-    }
-
-    public interface BlockingInterface {
-      public org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse test(
-          com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg request)
-          throws com.google.protobuf.ServiceException;
-    }
-
-    private static final class BlockingStub implements BlockingInterface {
-      private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) {
-        this.channel = channel;
-      }
-
-      private final com.google.protobuf.BlockingRpcChannel channel;
-
-      public org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse test(
-          com.google.protobuf.RpcController controller,
-          org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestArg request)
-          throws com.google.protobuf.ServiceException {
-        return (org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse) channel.callBlockingMethod(
-          getDescriptor().getMethods().get(0),
-          controller,
-          request,
-          org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos.TestResponse.getDefaultInstance());
-      }
-
-    }
-
-    // @@protoc_insertion_point(class_scope:TestDelayedService)
-  }
-
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_TestArg_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_TestArg_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_TestResponse_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_TestResponse_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\026test_delayed_rpc.proto\"\030\n\007TestArg\022\r\n\005d" +
-      "elay\030\001 \002(\010\" \n\014TestResponse\022\020\n\010response\030\001" +
-      " \002(\00525\n\022TestDelayedService\022\037\n\004test\022\010.Tes" +
-      "tArg\032\r.TestResponseBL\n.org.apache.hadoop" +
-      ".hbase.ipc.protobuf.generatedB\024TestDelay" +
-      "edRpcProtos\210\001\001\240\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_TestArg_descriptor =
-            getDescriptor().getMessageTypes().get(0);
-          internal_static_TestArg_fieldAccessorTable = new
-            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_TestArg_descriptor,
-              new java.lang.String[] { "Delay", });
-          internal_static_TestResponse_descriptor =
-            getDescriptor().getMessageTypes().get(1);
-          internal_static_TestResponse_fieldAccessorTable = new
-            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_TestResponse_descriptor,
-              new java.lang.String[] { "Response", });
-          return null;
-        }
-      };
-    com.google.protobuf.Descriptors.FileDescriptor
-      .internalBuildGeneratedFileFrom(descriptorData,
-        new com.google.protobuf.Descriptors.FileDescriptor[] {
-        }, assigner);
-  }
-
-  // @@protoc_insertion_point(outer_class_scope)
-}

http://git-wip-us.apache.org/repos/asf/hbase/blob/de62ad15/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestSecureIPC.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestSecureIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestSecureIPC.java
index 6145838..7e99cc0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestSecureIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/AbstractTestSecureIPC.java
@@ -32,20 +32,26 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Properties;
+import java.util.concurrent.ThreadLocalRandom;
 
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.ipc.FifoRpcScheduler;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
 import org.apache.hadoop.hbase.ipc.RpcClient;
 import org.apache.hadoop.hbase.ipc.RpcClientFactory;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.ipc.RpcServerInterface;
-import org.apache.hadoop.hbase.ipc.TestDelayedRpc.TestDelayedImplementation;
-import org.apache.hadoop.hbase.ipc.TestDelayedRpc.TestThread;
-import org.apache.hadoop.hbase.ipc.protobuf.generated.TestDelayedRpcProtos;
+import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos;
+import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
@@ -70,6 +76,55 @@ public abstract class AbstractTestSecureIPC {
   private static final File KEYTAB_FILE = new File(TEST_UTIL.getDataTestDir("keytab").toUri()
       .getPath());
 
+  static final BlockingService SERVICE =
+      TestRpcServiceProtos.TestProtobufRpcProto.newReflectiveBlockingService(
+          new TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface() {
+
+            @Override
+            public TestProtos.EmptyResponseProto ping(RpcController controller,
+                                                      TestProtos.EmptyRequestProto request)
+                throws ServiceException {
+              return null;
+            }
+
+            @Override
+            public TestProtos.EmptyResponseProto error(RpcController controller,
+                                                       TestProtos.EmptyRequestProto request)
+                throws ServiceException {
+              return null;
+            }
+
+            @Override
+            public TestProtos.EchoResponseProto echo(RpcController controller,
+                                                     TestProtos.EchoRequestProto request)
+                throws ServiceException {
+              if (controller instanceof PayloadCarryingRpcController) {
+                PayloadCarryingRpcController pcrc = (PayloadCarryingRpcController) controller;
+                // If cells, scan them to check we are able to iterate what we were given and since
+                // this is
+                // an echo, just put them back on the controller creating a new block. Tests our
+                // block
+                // building.
+                CellScanner cellScanner = pcrc.cellScanner();
+                List<Cell> list = null;
+                if (cellScanner != null) {
+                  list = new ArrayList<Cell>();
+                  try {
+                    while (cellScanner.advance()) {
+                      list.add(cellScanner.current());
+                    }
+                  } catch (IOException e) {
+                    throw new ServiceException(e);
+                  }
+                }
+                cellScanner = CellUtil.createCellScanner(list);
+                ((PayloadCarryingRpcController) controller).setCellScanner(cellScanner);
+              }
+              return TestProtos.EchoResponseProto.newBuilder()
+                  .setMessage(request.getMessage()).build();
+            }
+          });
+
   private static MiniKdc KDC;
   private static String HOST = "localhost";
   private static String PRINCIPAL;
@@ -189,36 +244,31 @@ public abstract class AbstractTestSecureIPC {
     SecurityInfo securityInfoMock = Mockito.mock(SecurityInfo.class);
     Mockito.when(securityInfoMock.getServerPrincipal())
         .thenReturn(HBaseKerberosUtils.KRB_PRINCIPAL);
-    SecurityInfo.addInfo("TestDelayedService", securityInfoMock);
+    SecurityInfo.addInfo("TestProtobufRpcProto", securityInfoMock);
 
-    boolean delayReturnValue = false;
     InetSocketAddress isa = new InetSocketAddress(HOST, 0);
-    TestDelayedImplementation instance = new TestDelayedImplementation(delayReturnValue);
-    BlockingService service =
-        TestDelayedRpcProtos.TestDelayedService.newReflectiveBlockingService(instance);
 
     RpcServerInterface rpcServer =
-        new RpcServer(null, "testSecuredDelayedRpc",
-            Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(service, null)), isa,
+        new RpcServer(null, "AbstractTestSecureIPC",
+            Lists.newArrayList(new RpcServer.BlockingServiceAndInterface(SERVICE, null)), isa,
             serverConf, new FifoRpcScheduler(serverConf, 1));
     rpcServer.start();
-    RpcClient rpcClient =
-        RpcClientFactory.createClient(clientConf, HConstants.DEFAULT_CLUSTER_ID.toString());
-    try {
+    try (RpcClient rpcClient = RpcClientFactory.createClient(clientConf,
+        HConstants.DEFAULT_CLUSTER_ID.toString())) {
       InetSocketAddress address = rpcServer.getListenerAddress();
       if (address == null) {
         throw new IOException("Listener channel is closed");
       }
       BlockingRpcChannel channel =
           rpcClient.createBlockingRpcChannel(
-            ServerName.valueOf(address.getHostName(), address.getPort(),
-            System.currentTimeMillis()), clientUser, 0);
-      TestDelayedRpcProtos.TestDelayedService.BlockingInterface stub =
-          TestDelayedRpcProtos.TestDelayedService.newBlockingStub(channel);
-      List<Integer> results = new ArrayList<>();
-      TestThread th1 = new TestThread(stub, true, results);
+              ServerName.valueOf(address.getHostName(), address.getPort(),
+                  System.currentTimeMillis()), clientUser, 0);
+      TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface stub =
+          TestRpcServiceProtos.TestProtobufRpcProto.newBlockingStub(channel);
+      List<String> results = new ArrayList<>();
+      TestThread th1 = new TestThread(stub, results);
       final Throwable exception[] = new Throwable[1];
-          Collections.synchronizedList(new ArrayList<Throwable>());
+      Collections.synchronizedList(new ArrayList<Throwable>());
       Thread.UncaughtExceptionHandler exceptionHandler =
           new Thread.UncaughtExceptionHandler() {
             public void uncaughtException(Thread th, Throwable ex) {
@@ -235,11 +285,35 @@ public abstract class AbstractTestSecureIPC {
         }
         throw (Exception) exception[0];
       }
-
-      assertEquals(0xDEADBEEF, results.get(0).intValue());
     } finally {
-      rpcClient.close();
       rpcServer.stop();
     }
   }
+
+  public static class TestThread extends Thread {
+      private final TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface stub;
+
+      private final List<String> results;
+
+          public TestThread(TestRpcServiceProtos.TestProtobufRpcProto.BlockingInterface stub, List<String> results) {
+          this.stub = stub;
+          this.results = results;
+        }
+
+          @Override
+      public void run() {
+          String result;
+          try {
+              result = stub.echo(null, TestProtos.EchoRequestProto.newBuilder().setMessage(String.valueOf(
+                  ThreadLocalRandom.current().nextInt())).build()).getMessage();
+            } catch (ServiceException e) {
+              throw new RuntimeException(e);
+            }
+          if (results != null) {
+              synchronized (results) {
+                  results.add(result);
+                }
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/de62ad15/hbase-server/src/test/protobuf/test_delayed_rpc.proto
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/protobuf/test_delayed_rpc.proto b/hbase-server/src/test/protobuf/test_delayed_rpc.proto
deleted file mode 100644
index cfab0fb..0000000
--- a/hbase-server/src/test/protobuf/test_delayed_rpc.proto
+++ /dev/null
@@ -1,34 +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.
- */
-option java_package = "org.apache.hadoop.hbase.ipc.protobuf.generated";
-option java_outer_classname = "TestDelayedRpcProtos";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-
-
-message TestArg {
-  required bool delay = 1;
-}
-
-message TestResponse {
-  required int32 response = 1;
-}
-
-service TestDelayedService {
-  rpc test(TestArg) returns (TestResponse);
-}