You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2016/11/01 19:49:54 UTC
[30/50] [abbrv] hbase git commit: HBASE-16752 Return error back to
rpc client on exceeding rpc request size limit.
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/33e89fa9
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/33e89fa9
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/33e89fa9
Branch: refs/heads/branch-1
Commit: 33e89fa9cfd4036fe972ff892df957603ae25bbf
Parents: a97aef5
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:50:42 2016 -0700
----------------------------------------------------------------------
.../hbase/exceptions/ClientExceptionsUtil.java | 2 +-
.../exceptions/RequestTooBigException.java | 43 +++++++++++++++++
.../org/apache/hadoop/hbase/ipc/RpcServer.java | 49 ++++++++++++++++++--
.../hadoop/hbase/ipc/AbstractTestIPC.java | 17 ++++---
4 files changed, 101 insertions(+), 10 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/33e89fa9/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/33e89fa9/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/33e89fa9/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 90a53bc..9c21bbe 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
@@ -24,6 +24,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.BoundedByteBufferPool;
import org.apache.hadoop.hbase.io.ByteBufferInputStream;
import org.apache.hadoop.hbase.io.ByteBufferOutputStream;
@@ -263,6 +265,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";
@@ -1653,9 +1658,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.
+ final 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/33e89fa9/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 69c8fe2..bbf8720 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
@@ -300,15 +300,18 @@ public abstract class AbstractTestIPC {
@Test
public void testRpcMaxRequestSize() throws IOException, InterruptedException {
Configuration conf = new Configuration(CONF);
- conf.setInt(RpcServer.MAX_REQUEST_SIZE, 100);
+ conf.setInt(RpcServer.MAX_REQUEST_SIZE, 1000);
RpcServer rpcServer = new TestRpcServer(conf);
AbstractRpcClient client = createRpcClient(conf);
try {
rpcServer.start();
MethodDescriptor md = SERVICE.getDescriptorForType().findMethodByName("echo");
- // set total RPC size bigger than 100 bytes
- EchoRequestProto param = EchoRequestProto.newBuilder().setMessage("hello.hello.hello.hello."
- + "hello.hello.hello.hello.hello.hello.hello.hello.hello.hello.hello.hello").build();
+ StringBuilder message = new StringBuilder(1200);
+ for (int i = 0; i < 200; i++) {
+ message.append("hello.");
+ }
+ // set total RPC size bigger than 1000 bytes
+ EchoRequestProto param = EchoRequestProto.newBuilder().setMessage(message.toString()).build();
InetSocketAddress address = rpcServer.getListenerAddress();
if (address == null) {
throw new IOException("Listener channel is closed");
@@ -319,8 +322,10 @@ public abstract class AbstractTestIPC {
md.getOutputType().toProto(), User.getCurrent(), address,
new MetricsConnection.CallStats());
fail("RPC should have failed because it exceeds max request size");
- } catch(IOException ex) {
- // pass
+ } catch(IOException e) {
+ LOG.info("Caught expected exception: " + e);
+ assertTrue(e.toString(),
+ StringUtils.stringifyException(e).contains("RequestTooBigException"));
}
} finally {
rpcServer.stop();