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

hbase git commit: HBASE-16752 Return error back to rpc client on exceeding rpc request size limit.

Repository: hbase
Updated Branches:
  refs/heads/master 674511875 -> ee1123b06


HBASE-16752 Return error back to rpc client on exceeding rpc request size limit.

Signed-off-by: Gary Helmling <ga...@apache.org>


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

Branch: refs/heads/master
Commit: ee1123b0696e290fe406e3b7de30052f6dd06ba7
Parents: 6745118
Author: Ashu Pachauri <as...@gmail.com>
Authored: Thu Oct 6 17:09:19 2016 -0700
Committer: Gary Helmling <ga...@apache.org>
Committed: Wed Oct 19 15:09:51 2016 -0700

----------------------------------------------------------------------
 .../hbase/exceptions/ClientExceptionsUtil.java  |  2 +-
 .../exceptions/RequestTooBigException.java      | 43 +++++++++++++++++
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  | 49 ++++++++++++++++++--
 .../hadoop/hbase/ipc/AbstractTestIPC.java       | 11 +++--
 4 files changed, 96 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/ee1123b0/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
index f367ed9..8acfe49 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/ClientExceptionsUtil.java
@@ -63,7 +63,7 @@ public final class ClientExceptionsUtil {
         || cur instanceof RegionTooBusyException || cur instanceof ThrottlingException
         || cur instanceof MultiActionResultTooLarge || cur instanceof RetryImmediatelyException
         || cur instanceof CallQueueTooBigException || cur instanceof CallDroppedException
-        || cur instanceof NotServingRegionException);
+        || cur instanceof NotServingRegionException || cur instanceof RequestTooBigException);
   }
 
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/ee1123b0/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java
new file mode 100644
index 0000000..31baebb
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/RequestTooBigException.java
@@ -0,0 +1,43 @@
+/**
+ * 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.exceptions;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.classification.InterfaceStability;
+
+/**
+ * Thrown when the size of the rpc request received by the server is too large.
+ *
+ * On receiving such an exception, the client does not retry the offending rpc.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class RequestTooBigException extends DoNotRetryIOException {
+
+  private static final long serialVersionUID = -1593339239809586516L;
+
+  public RequestTooBigException() {
+    super();
+  }
+
+  public RequestTooBigException(String message) {
+    super(message);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/ee1123b0/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 b026475..542510c 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
@@ -26,6 +26,7 @@ import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.net.BindException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
@@ -85,6 +86,7 @@ import org.apache.hadoop.hbase.client.VersionInfoUtil;
 import org.apache.hadoop.hbase.codec.Codec;
 import org.apache.hadoop.hbase.conf.ConfigurationObserver;
 import org.apache.hadoop.hbase.exceptions.RegionMovedException;
+import org.apache.hadoop.hbase.exceptions.RequestTooBigException;
 import org.apache.hadoop.hbase.io.ByteBufferInputStream;
 import org.apache.hadoop.hbase.io.ByteBufferListOutputStream;
 import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
@@ -257,6 +259,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
   protected HBaseRPCErrorHandler errorHandler = null;
 
   static final String MAX_REQUEST_SIZE = "hbase.ipc.max.request.size";
+  private static final RequestTooBigException REQUEST_TOO_BIG_EXCEPTION =
+      new RequestTooBigException();
+
   private static final String WARN_RESPONSE_TIME = "hbase.ipc.warn.response.time";
   private static final String WARN_RESPONSE_SIZE = "hbase.ipc.warn.response.size";
 
@@ -1621,9 +1626,47 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
         }
 
         if (dataLength > maxRequestSize) {
-          throw new DoNotRetryIOException("RPC data length of " + dataLength + " received from "
-              + getHostAddress() + " is greater than max allowed " + maxRequestSize + ". Set \""
-              + MAX_REQUEST_SIZE + "\" on server to override this limit (not recommended)");
+          String msg = "RPC data length of " + dataLength + " received from "
+              + getHostAddress() + " is greater than max allowed "
+              + maxRequestSize + ". Set \"" + MAX_REQUEST_SIZE
+              + "\" on server to override this limit (not recommended)";
+          LOG.warn(msg);
+
+          if (connectionHeaderRead && connectionPreambleRead) {
+            incRpcCount();
+            // Construct InputStream for the non-blocking SocketChannel
+            // We need the InputStream because we want to read only the request header
+            // instead of the whole rpc.
+            ByteBuffer buf = ByteBuffer.allocate(1);
+            InputStream is = new InputStream() {
+              @Override
+              public int read() throws IOException {
+                channelRead(channel, buf);
+                buf.flip();
+                int x = buf.get();
+                buf.flip();
+                return x;
+              }
+            };
+            CodedInputStream cis = CodedInputStream.newInstance(is);
+            int headerSize = cis.readRawVarint32();
+            Message.Builder builder = RequestHeader.newBuilder();
+            ProtobufUtil.mergeFrom(builder, cis, headerSize);
+            RequestHeader header = (RequestHeader) builder.build();
+
+            // Notify the client about the offending request
+            Call reqTooBig = new Call(header.getCallId(), this.service, null, null, null,
+                null, this, responder, 0, null, this.addr,0);
+            metrics.exception(REQUEST_TOO_BIG_EXCEPTION);
+            setupResponse(null, reqTooBig, REQUEST_TOO_BIG_EXCEPTION, msg);
+            // We are going to close the connection, make sure we process the response
+            // before that. In rare case when this fails, we still close the connection.
+            responseWriteLock.lock();
+            responder.processResponse(reqTooBig);
+            responseWriteLock.unlock();
+          }
+          // Close the connection
+          return -1;
         }
 
         data = ByteBuffer.allocate(dataLength);

http://git-wip-us.apache.org/repos/asf/hbase/blob/ee1123b0/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
index 652ba3f..2211e8f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/AbstractTestIPC.java
@@ -204,13 +204,13 @@ public abstract class AbstractTestIPC {
   @Test
   public void testRpcMaxRequestSize() throws IOException, ServiceException {
     Configuration conf = new Configuration(CONF);
-    conf.setInt(RpcServer.MAX_REQUEST_SIZE, 100);
+    conf.setInt(RpcServer.MAX_REQUEST_SIZE, 1000);
     RpcServer rpcServer = new TestRpcServer(conf);
     try (AbstractRpcClient<?> client = createRpcClient(conf)) {
       rpcServer.start();
       BlockingInterface stub = newBlockingStub(client, rpcServer.getListenerAddress());
-      StringBuilder message = new StringBuilder(120);
-      for (int i = 0; i < 20; i++) {
+      StringBuilder message = new StringBuilder(1200);
+      for (int i = 0; i < 200; i++) {
         message.append("hello.");
       }
       // set total RPC size bigger than 100 bytes
@@ -220,8 +220,9 @@ public abstract class AbstractTestIPC {
         param);
       fail("RPC should have failed because it exceeds max request size");
     } catch (ServiceException e) {
-      LOG.info("Caught expected exception: " + e.toString());
-      // the rpc server just close the connection so we can not get the detail message.
+      LOG.info("Caught expected exception: " + e);
+      assertTrue(e.toString(),
+          StringUtils.stringifyException(e).contains("RequestTooBigException"));
     } finally {
       rpcServer.stop();
     }