You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2020/12/20 04:00:26 UTC
[hbase] branch master updated: HBASE-25419 Remove deprecated
methods in RpcServer implementation (#2791)
This is an automated email from the ASF dual-hosted git repository.
zhangduo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/master by this push:
new 772350b HBASE-25419 Remove deprecated methods in RpcServer implementation (#2791)
772350b is described below
commit 772350b26e9db75bf5b7c7bc59e817a3e39d637b
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Sun Dec 20 12:00:07 2020 +0800
HBASE-25419 Remove deprecated methods in RpcServer implementation (#2791)
Signed-off-by: XinSun <dd...@gmail.com>
Signed-off-by: stack <st...@apache.com>
---
.../apache/hadoop/hbase/ipc/NettyRpcServer.java | 23 ----------------------
.../hadoop/hbase/ipc/RpcServerInterface.java | 20 +------------------
.../apache/hadoop/hbase/ipc/SimpleRpcServer.java | 23 ----------------------
3 files changed, 1 insertion(+), 65 deletions(-)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
index f34cad5..1d3981f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/NettyRpcServer.java
@@ -23,22 +23,16 @@ import java.net.InetSocketAddress;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.security.HBasePolicyProvider;
import org.apache.hadoop.hbase.util.NettyEventLoopGroupConfig;
-import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
-import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
-import org.apache.hbase.thirdparty.com.google.protobuf.Message;
import org.apache.hbase.thirdparty.io.netty.bootstrap.ServerBootstrap;
import org.apache.hbase.thirdparty.io.netty.channel.Channel;
import org.apache.hbase.thirdparty.io.netty.channel.ChannelInitializer;
@@ -182,21 +176,4 @@ public class NettyRpcServer extends RpcServer {
// allChannels also contains the server channel, so exclude that from the count.
return channelsCount > 0 ? channelsCount - 1 : channelsCount;
}
-
- @Override
- public Pair<Message, CellScanner> call(BlockingService service,
- MethodDescriptor md, Message param, CellScanner cellScanner,
- long receiveTime, MonitoredRPCHandler status) throws IOException {
- return call(service, md, param, cellScanner, receiveTime, status,
- System.currentTimeMillis(), 0);
- }
-
- @Override
- public Pair<Message, CellScanner> call(BlockingService service, MethodDescriptor md,
- Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status,
- long startTime, int timeout) throws IOException {
- NettyServerCall fakeCall = new NettyServerCall(-1, service, md, null, param, cellScanner, null,
- -1, null, receiveTime, timeout, bbAllocator, cellBlockBuilder, null);
- return call(fakeCall, status);
- }
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java
index e06daac..ee6e57a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServerInterface.java
@@ -25,14 +25,12 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.io.ByteBuffAllocator;
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
-import org.apache.hadoop.hbase.regionserver.RSRpcServices;
import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
+import org.apache.hadoop.hbase.regionserver.RSRpcServices;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
-import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
@InterfaceAudience.Private
@@ -46,22 +44,6 @@ public interface RpcServerInterface {
void setSocketSendBufSize(int size);
InetSocketAddress getListenerAddress();
- /**
- * @deprecated As of release 1.3, this will be removed in HBase 3.0
- */
- @Deprecated
- Pair<Message, CellScanner> call(BlockingService service, MethodDescriptor md,
- Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status)
- throws IOException;
-
- /**
- * @deprecated As of release 2.0, this will be removed in HBase 3.0
- */
- @Deprecated
- Pair<Message, CellScanner> call(BlockingService service, MethodDescriptor md, Message param,
- CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status, long startTime,
- int timeout) throws IOException;
-
Pair<Message, CellScanner> call(RpcCall call, MonitoredRPCHandler status)
throws IOException;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcServer.java
index cbcbc9a..38c7712 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcServer.java
@@ -41,22 +41,16 @@ import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.Server;
-import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
import org.apache.hadoop.hbase.security.HBasePolicyProvider;
-import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
-import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
-import org.apache.hbase.thirdparty.com.google.protobuf.Message;
/**
* The RPC server with native java NIO implementation deriving from Hadoop to
@@ -475,23 +469,6 @@ public class SimpleRpcServer extends RpcServer {
return listener.getAddress();
}
- @Override
- public Pair<Message, CellScanner> call(BlockingService service, MethodDescriptor md,
- Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status)
- throws IOException {
- return call(service, md, param, cellScanner, receiveTime, status, System.currentTimeMillis(),
- 0);
- }
-
- @Override
- public Pair<Message, CellScanner> call(BlockingService service, MethodDescriptor md,
- Message param, CellScanner cellScanner, long receiveTime, MonitoredRPCHandler status,
- long startTime, int timeout) throws IOException {
- SimpleServerCall fakeCall = new SimpleServerCall(-1, service, md, null, param, cellScanner,
- null, -1, null, receiveTime, timeout, bbAllocator, cellBlockBuilder, null, null);
- return call(fakeCall, status);
- }
-
/**
* This is a wrapper around {@link java.nio.channels.WritableByteChannel#write(java.nio.ByteBuffer)}.
* If the amount of data is large, it writes to channel in smaller chunks.