HBASE-11497 Expose RpcScheduling implementations as LimitedPrivate interfaces (Jesse Yates)
This commit is contained in:
parent
2b75a0bd63
commit
39100cb1d1
|
@ -27,4 +27,5 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
@InterfaceStability.Evolving
|
||||
public class HBaseInterfaceAudience {
|
||||
public static final String COPROC = "Coprocesssor";
|
||||
public static final String PHOENIX = "Phoenix";
|
||||
}
|
||||
|
|
|
@ -18,27 +18,22 @@
|
|||
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* RPC Executor that dispatch the requests on multiple queues.
|
||||
* Each handler has its own queue and there is no stealing.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
|
||||
@InterfaceStability.Evolving
|
||||
public class MultipleQueueRpcExecutor extends RpcExecutor {
|
||||
protected final List<BlockingQueue<CallRunner>> queues;
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
@ -31,20 +29,20 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.protobuf.Message;
|
||||
|
||||
/**
|
||||
* RPC Executor that uses different queues for reads and writes.
|
||||
* Each handler has its own queue and there is no stealing.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
|
||||
@InterfaceStability.Evolving
|
||||
public class RWQueueRpcExecutor extends RpcExecutor {
|
||||
private static final Log LOG = LogFactory.getLog(RWQueueRpcExecutor.class);
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.ipc;
|
|||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
|
@ -26,7 +27,7 @@ import java.net.InetSocketAddress;
|
|||
/**
|
||||
* An interface for RPC request scheduling algorithm.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
|
||||
@InterfaceStability.Evolving
|
||||
public interface RpcScheduler {
|
||||
|
||||
|
|
|
@ -64,13 +64,16 @@ import javax.security.sasl.SaslException;
|
|||
import javax.security.sasl.SaslServer;
|
||||
|
||||
import io.netty.util.internal.ConcurrentSet;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseIOException;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
|
@ -148,7 +151,8 @@ import com.google.protobuf.TextFormat;
|
|||
*
|
||||
* @see RpcClient
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
|
||||
@InterfaceStability.Evolving
|
||||
public class RpcServer implements RpcServerInterface {
|
||||
public static final Log LOG = LogFactory.getLog(RpcServer.class);
|
||||
|
||||
|
|
|
@ -23,7 +23,9 @@ import java.io.IOException;
|
|||
import java.net.InetSocketAddress;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.security.authorize.PolicyProvider;
|
||||
|
@ -34,7 +36,8 @@ import com.google.protobuf.Descriptors.MethodDescriptor;
|
|||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
|
||||
@InterfaceStability.Evolving
|
||||
public interface RpcServerInterface {
|
||||
void start();
|
||||
boolean isStarted();
|
||||
|
|
|
@ -17,32 +17,23 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import java.util.Random;
|
||||
import java.util.Comparator;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.util.BoundedPriorityBlockingQueue;
|
||||
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* A scheduler that maintains isolated handler pools for general,
|
||||
* high-priority, and replication requests.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
|
||||
@InterfaceStability.Evolving
|
||||
public class SimpleRpcScheduler implements RpcScheduler {
|
||||
public static final Log LOG = LogFactory.getLog(SimpleRpcScheduler.class);
|
||||
|
|
|
@ -18,25 +18,20 @@
|
|||
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* RPC Executor that uses a single queue for all the requests.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
|
||||
@InterfaceStability.Evolving
|
||||
public class SingleQueueRpcExecutor extends RpcExecutor {
|
||||
private final BlockingQueue<CallRunner> queue;
|
||||
|
|
|
@ -17,14 +17,19 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.ipc.PriorityFunction;
|
||||
import org.apache.hadoop.hbase.ipc.RpcScheduler;
|
||||
import org.apache.hadoop.hbase.ipc.SimpleRpcScheduler;
|
||||
|
||||
/** Constructs a {@link SimpleRpcScheduler}. */
|
||||
class SimpleRpcSchedulerFactory implements RpcSchedulerFactory {
|
||||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
|
||||
@InterfaceStability.Evolving
|
||||
public class SimpleRpcSchedulerFactory implements RpcSchedulerFactory {
|
||||
|
||||
@Override
|
||||
public RpcScheduler create(Configuration conf, PriorityFunction priority) {
|
||||
|
|
Loading…
Reference in New Issue