HBASE-23799 Make our core coprocessors use shaded protobuf (#1280)
Signed-off-by: stack <stack@apache.org>
This commit is contained in:
parent
542244056c
commit
eface74407
|
@ -21,11 +21,11 @@ package org.apache.hadoop.hbase;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
|
||||
import com.google.protobuf.Service;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
/**
|
||||
* Base interface for the 4 coprocessors - MasterCoprocessor, RegionCoprocessor,
|
||||
* RegionServerCoprocessor, and WALCoprocessor.
|
||||
|
|
|
@ -65,11 +65,6 @@ import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
|||
import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.ExceptionUtil;
|
||||
|
@ -82,6 +77,12 @@ import org.slf4j.LoggerFactory;
|
|||
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Read/write operations on <code>hbase:meta</code> region as well as assignment information stored
|
||||
|
|
|
@ -20,12 +20,6 @@ package org.apache.hadoop.hbase.client;
|
|||
import static org.apache.hadoop.hbase.client.ConnectionUtils.setCoprocessorError;
|
||||
import static org.apache.hadoop.hbase.util.FutureUtils.get;
|
||||
|
||||
import com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcChannel;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.EnumSet;
|
||||
|
@ -71,6 +65,13 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
|
||||
/**
|
||||
* The {@link Admin} implementation which is based on an {@link AsyncAdmin}.
|
||||
*/
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
|
||||
|
||||
import com.google.protobuf.RpcChannel;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
@ -54,6 +53,8 @@ import org.apache.hadoop.hbase.security.access.UserPermission;
|
|||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
|
||||
|
||||
/**
|
||||
* The asynchronous administrative API for HBase.
|
||||
* @since 2.0.0
|
||||
|
|
|
@ -17,8 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import com.google.protobuf.RpcChannel;
|
||||
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -52,6 +50,8 @@ import org.apache.hadoop.hbase.util.FutureUtils;
|
|||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
|
||||
|
||||
/**
|
||||
* Just a wrapper of {@link RawAsyncHBaseAdmin}. The difference is that users need to provide a
|
||||
* thread pool when constructing this class, and the callback methods registered to the returned
|
||||
|
|
|
@ -21,7 +21,6 @@ import static java.util.stream.Collectors.toList;
|
|||
import static org.apache.hadoop.hbase.client.ConnectionUtils.allOf;
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.toCheckExistenceOnly;
|
||||
|
||||
import com.google.protobuf.RpcChannel;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -35,6 +34,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
|
||||
|
||||
/**
|
||||
* The interface for asynchronous version of Table. Obtain an instance from a
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
import com.google.protobuf.RpcChannel;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
|
@ -34,6 +33,8 @@ import org.apache.hadoop.hbase.io.TimeRange;
|
|||
import org.apache.hadoop.hbase.util.FutureUtils;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
|
||||
|
||||
/**
|
||||
* Just a wrapper of {@link RawAsyncTableImpl}. The difference is that users need to provide a
|
||||
* thread pool when constructing this class, and the callback methods registered to the returned
|
||||
|
|
|
@ -17,11 +17,12 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
|
||||
/**
|
||||
* Client side rpc controller for coprocessor implementation. It is only used to pass error.
|
||||
*/
|
||||
|
|
|
@ -59,6 +59,7 @@ import org.slf4j.LoggerFactory;
|
|||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
import org.apache.hbase.thirdparty.io.netty.util.Timer;
|
||||
|
||||
|
@ -653,7 +654,7 @@ public final class ConnectionUtils {
|
|||
}
|
||||
}
|
||||
|
||||
static void setCoprocessorError(com.google.protobuf.RpcController controller, Throwable error) {
|
||||
static void setCoprocessorError(RpcController controller, Throwable error) {
|
||||
if (controller == null) {
|
||||
return;
|
||||
}
|
||||
|
|
|
@ -17,11 +17,12 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
|
||||
/**
|
||||
* For implementation coprocessor related methods in {@link Table} and {@link Admin} interface.
|
||||
* @deprecated since 3.0.0, will be removed in 4.0.0 along with the coprocessor related methods in
|
||||
|
|
|
@ -19,11 +19,6 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
|
||||
|
||||
import com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcChannel;
|
||||
import com.google.protobuf.RpcController;
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.MasterRequestCallerBuilder;
|
||||
|
@ -31,6 +26,12 @@ import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
|||
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
|
||||
|
|
|
@ -47,8 +47,6 @@ import org.apache.hadoop.hbase.RawCell;
|
|||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.io.HeapSize;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.security.access.AccessControlConstants;
|
||||
import org.apache.hadoop.hbase.security.access.AccessControlUtil;
|
||||
import org.apache.hadoop.hbase.security.access.Permission;
|
||||
|
@ -65,6 +63,9 @@ import org.apache.hbase.thirdparty.com.google.common.io.ByteArrayDataInput;
|
|||
import org.apache.hbase.thirdparty.com.google.common.io.ByteArrayDataOutput;
|
||||
import org.apache.hbase.thirdparty.com.google.common.io.ByteStreams;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
|
||||
|
||||
@InterfaceAudience.Public
|
||||
public abstract class Mutation extends OperationWithAttributes implements Row, CellScannable,
|
||||
HeapSize {
|
||||
|
|
|
@ -21,8 +21,7 @@ import static org.apache.hadoop.hbase.HConstants.HIGH_QOS;
|
|||
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
|
||||
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
|
||||
import static org.apache.hadoop.hbase.util.FutureUtils.unwrapCompletionException;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcChannel;
|
||||
|
||||
import edu.umd.cs.findbugs.annotations.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -46,7 +45,6 @@ import java.util.function.Supplier;
|
|||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.AsyncMetaTableAccessor;
|
||||
import org.apache.hadoop.hbase.CacheEvictionStats;
|
||||
|
@ -102,9 +100,12 @@ import org.apache.hadoop.hbase.util.Pair;
|
|||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
|
||||
import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
|
||||
import org.apache.hbase.thirdparty.io.netty.util.Timeout;
|
||||
import org.apache.hbase.thirdparty.io.netty.util.TimerTask;
|
||||
|
|
|
@ -24,7 +24,6 @@ import static org.apache.hadoop.hbase.client.ConnectionUtils.timelineConsistentR
|
|||
import static org.apache.hadoop.hbase.client.ConnectionUtils.validatePut;
|
||||
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
|
||||
|
||||
import com.google.protobuf.RpcChannel;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
|
@ -52,6 +51,7 @@ import org.slf4j.LoggerFactory;
|
|||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
|
||||
import org.apache.hbase.thirdparty.io.netty.util.Timer;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
|
|
|
@ -20,11 +20,6 @@ package org.apache.hadoop.hbase.client;
|
|||
import static org.apache.hadoop.hbase.client.ConnectionUtils.setCoprocessorError;
|
||||
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
|
||||
|
||||
import com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcChannel;
|
||||
import com.google.protobuf.RpcController;
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
@ -36,6 +31,12 @@ import org.apache.hadoop.hbase.ipc.HBaseRpcController;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
|
||||
|
|
|
@ -19,12 +19,12 @@
|
|||
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Objects;
|
||||
import com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
|
||||
|
||||
/**
|
||||
|
|
|
@ -19,11 +19,6 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
|
||||
|
||||
import com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcChannel;
|
||||
import com.google.protobuf.RpcController;
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import org.apache.hadoop.hbase.client.AsyncRpcRetryingCallerFactory.ServerRequestCallerBuilder;
|
||||
|
@ -31,6 +26,12 @@ import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
|||
import org.apache.hadoop.hbase.ipc.HBaseRpcController;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
|
||||
|
|
|
@ -17,11 +17,11 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
|
||||
/**
|
||||
* Delegate to a protobuf rpc call.
|
||||
* <p>
|
||||
|
|
|
@ -17,19 +17,18 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
|
||||
/**
|
||||
* Base class which provides clients with an RPC connection to
|
||||
|
@ -46,10 +45,8 @@ abstract class SyncCoprocessorRpcChannel implements CoprocessorRpcChannel {
|
|||
|
||||
@Override
|
||||
@InterfaceAudience.Private
|
||||
public void callMethod(Descriptors.MethodDescriptor method,
|
||||
RpcController controller,
|
||||
Message request, Message responsePrototype,
|
||||
RpcCallback<Message> callback) {
|
||||
public void callMethod(Descriptors.MethodDescriptor method, RpcController controller,
|
||||
Message request, Message responsePrototype, RpcCallback<Message> callback) {
|
||||
Message response = null;
|
||||
try {
|
||||
response = callExecService(controller, method, request, responsePrototype);
|
||||
|
@ -64,14 +61,12 @@ abstract class SyncCoprocessorRpcChannel implements CoprocessorRpcChannel {
|
|||
|
||||
@Override
|
||||
@InterfaceAudience.Private
|
||||
public Message callBlockingMethod(Descriptors.MethodDescriptor method,
|
||||
RpcController controller,
|
||||
Message request, Message responsePrototype)
|
||||
throws ServiceException {
|
||||
public Message callBlockingMethod(Descriptors.MethodDescriptor method, RpcController controller,
|
||||
Message request, Message responsePrototype) throws ServiceException {
|
||||
try {
|
||||
return callExecService(controller, method, request, responsePrototype);
|
||||
} catch (IOException ioe) {
|
||||
throw new ServiceException("Error calling method "+method.getFullName(), ioe);
|
||||
throw new ServiceException("Error calling method " + method.getFullName(), ioe);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -18,10 +18,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client;
|
||||
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.Service;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
|
@ -42,6 +38,11 @@ import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
|
||||
/**
|
||||
* Used to communicate with a single HBase table.
|
||||
* Obtain an instance from a {@link Connection} and call {@link #close()} afterwards.
|
||||
|
|
|
@ -19,12 +19,6 @@ package org.apache.hadoop.hbase.client;
|
|||
|
||||
import static org.apache.hadoop.hbase.client.ConnectionUtils.setCoprocessorError;
|
||||
|
||||
import com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -65,6 +59,14 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.primitives.Booleans;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
|
||||
/**
|
||||
* The table implementation based on {@link AsyncTable}.
|
||||
|
@ -493,7 +495,7 @@ class TableOverAsyncTable implements Table {
|
|||
public <T extends Service, R> void coprocessorService(Class<T> service, byte[] startKey,
|
||||
byte[] endKey, Call<T, R> callable, Callback<R> callback) throws ServiceException, Throwable {
|
||||
coprocssorService(service.getName(), startKey, endKey, callback, channel -> {
|
||||
T instance = org.apache.hadoop.hbase.protobuf.ProtobufUtil.newServiceStub(service, channel);
|
||||
T instance = ProtobufUtil.newServiceStub(service, channel);
|
||||
return callable.call(instance);
|
||||
});
|
||||
}
|
||||
|
|
|
@ -21,18 +21,19 @@ package org.apache.hadoop.hbase.client.coprocessor;
|
|||
import java.io.IOException;
|
||||
import java.math.BigDecimal;
|
||||
import java.math.RoundingMode;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BigDecimalMsg;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.BigDecimalMsg;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg;
|
||||
|
||||
/**
|
||||
* ColumnInterpreter for doing Aggregation's with BigDecimal columns. This class
|
||||
|
@ -124,7 +125,7 @@ public class BigDecimalColumnInterpreter extends ColumnInterpreter<BigDecimal, B
|
|||
|
||||
private BigDecimalMsg getProtoForType(BigDecimal t) {
|
||||
BigDecimalMsg.Builder builder = BigDecimalMsg.newBuilder();
|
||||
return builder.setBigdecimalMsg(ByteStringer.wrap(Bytes.toBytes(t))).build();
|
||||
return builder.setBigdecimalMsg(UnsafeByteOperations.unsafeWrap(Bytes.toBytes(t))).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -19,16 +19,16 @@
|
|||
package org.apache.hadoop.hbase.client.coprocessor;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.DoubleMsg;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.DoubleMsg;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg;
|
||||
|
||||
/**
|
||||
* a concrete column interpreter implementation. The cell value is a Double value
|
||||
|
|
|
@ -19,16 +19,16 @@
|
|||
package org.apache.hadoop.hbase.client.coprocessor;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.EmptyMsg;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.LongMsg;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.EmptyMsg;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.LongMsg;
|
||||
|
||||
/**
|
||||
* a concrete column interpreter implementation. The cell value is a Long value
|
||||
|
|
|
@ -20,13 +20,12 @@
|
|||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
import com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
|
||||
/**
|
||||
* Defines how value for specific column is interpreted and provides utility
|
||||
|
|
|
@ -19,13 +19,15 @@
|
|||
|
||||
package org.apache.hadoop.hbase.filter;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.InvalidProtocolBufferException;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ComparatorProtos;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ComparatorProtos;
|
||||
|
||||
/**
|
||||
* A comparator which compares against a specified byte array, but only
|
||||
* compares specific portion of the byte array. For the rest it is similar to
|
||||
|
|
|
@ -17,10 +17,11 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import com.google.protobuf.BlockingRpcChannel;
|
||||
import com.google.protobuf.RpcChannel;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcChannel;
|
||||
|
||||
/**
|
||||
* Base interface which provides clients with an RPC connection to call coprocessor endpoint
|
||||
* {@link com.google.protobuf.Service}s.
|
||||
|
|
|
@ -20,31 +20,33 @@
|
|||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import static org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME;
|
||||
|
||||
import edu.umd.cs.findbugs.annotations.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.ServiceDescriptor;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import com.google.protobuf.Descriptors.ServiceDescriptor;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.Service;
|
||||
|
||||
import edu.umd.cs.findbugs.annotations.Nullable;
|
||||
|
||||
/**
|
||||
* Utilities for handling coprocessor rpc service calls.
|
||||
|
@ -87,53 +89,47 @@ public final class CoprocessorRpcUtils {
|
|||
}
|
||||
|
||||
public static CoprocessorServiceRequest getCoprocessorServiceRequest(
|
||||
final Descriptors.MethodDescriptor method, final Message request, final byte [] row,
|
||||
final byte [] regionName) {
|
||||
return CoprocessorServiceRequest.newBuilder().setCall(
|
||||
getCoprocessorServiceCall(method, request, row)).
|
||||
setRegion(RequestConverter.buildRegionSpecifier(REGION_NAME, regionName)).build();
|
||||
final Descriptors.MethodDescriptor method, final Message request, final byte[] row,
|
||||
final byte[] regionName) {
|
||||
return CoprocessorServiceRequest.newBuilder()
|
||||
.setCall(getCoprocessorServiceCall(method, request, row))
|
||||
.setRegion(RequestConverter.buildRegionSpecifier(REGION_NAME, regionName)).build();
|
||||
}
|
||||
|
||||
private static CoprocessorServiceCall getCoprocessorServiceCall(
|
||||
final Descriptors.MethodDescriptor method, final Message request, final byte [] row) {
|
||||
private static CoprocessorServiceCall getCoprocessorServiceCall(final MethodDescriptor method,
|
||||
final Message request, final byte[] row) {
|
||||
return CoprocessorServiceCall.newBuilder()
|
||||
.setRow(org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations.unsafeWrap(row))
|
||||
.setServiceName(CoprocessorRpcUtils.getServiceName(method.getService()))
|
||||
.setMethodName(method.getName())
|
||||
// TODO!!!!! Come back here after!!!!! This is a double copy of the request if I read
|
||||
// it right copying from non-shaded to shaded version!!!!!! FIXXXXX!!!!!
|
||||
.setRequest(org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations.
|
||||
unsafeWrap(request.toByteArray())).build();
|
||||
.setRow(org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations.unsafeWrap(row))
|
||||
.setServiceName(CoprocessorRpcUtils.getServiceName(method.getService()))
|
||||
.setMethodName(method.getName())
|
||||
// TODO!!!!! Come back here after!!!!! This is a double copy of the request if I read
|
||||
// it right copying from non-shaded to shaded version!!!!!! FIXXXXX!!!!!
|
||||
.setRequest(org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations
|
||||
.unsafeWrap(request.toByteArray()))
|
||||
.build();
|
||||
}
|
||||
|
||||
public static MethodDescriptor getMethodDescriptor(final String methodName,
|
||||
final ServiceDescriptor serviceDesc)
|
||||
throws UnknownProtocolException {
|
||||
Descriptors.MethodDescriptor methodDesc = serviceDesc.findMethodByName(methodName);
|
||||
final ServiceDescriptor serviceDesc) throws UnknownProtocolException {
|
||||
MethodDescriptor methodDesc = serviceDesc.findMethodByName(methodName);
|
||||
if (methodDesc == null) {
|
||||
throw new UnknownProtocolException("Unknown method " + methodName + " called on service " +
|
||||
serviceDesc.getFullName());
|
||||
throw new UnknownProtocolException(
|
||||
"Unknown method " + methodName + " called on service " + serviceDesc.getFullName());
|
||||
}
|
||||
return methodDesc;
|
||||
}
|
||||
|
||||
public static Message getRequest(Service service,
|
||||
Descriptors.MethodDescriptor methodDesc,
|
||||
org.apache.hbase.thirdparty.com.google.protobuf.ByteString shadedRequest)
|
||||
throws IOException {
|
||||
Message.Builder builderForType =
|
||||
service.getRequestPrototype(methodDesc).newBuilderForType();
|
||||
org.apache.hadoop.hbase.protobuf.ProtobufUtil.mergeFrom(builderForType,
|
||||
// TODO: COPY FROM SHADED TO NON_SHADED. DO I HAVE TOO?
|
||||
shadedRequest.toByteArray());
|
||||
public static Message getRequest(Service service, MethodDescriptor methodDesc,
|
||||
ByteString shadedRequest) throws IOException {
|
||||
Message.Builder builderForType = service.getRequestPrototype(methodDesc).newBuilderForType();
|
||||
ProtobufUtil.mergeFrom(builderForType,
|
||||
// TODO: COPY FROM SHADED TO NON_SHADED. DO I HAVE TOO?
|
||||
shadedRequest.toByteArray());
|
||||
return builderForType.build();
|
||||
}
|
||||
|
||||
public static Message getResponse(
|
||||
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse
|
||||
result,
|
||||
com.google.protobuf.Message responsePrototype)
|
||||
throws IOException {
|
||||
public static Message getResponse(ClientProtos.CoprocessorServiceResponse result,
|
||||
Message responsePrototype) throws IOException {
|
||||
Message response;
|
||||
if (result.getValue().hasValue()) {
|
||||
Message.Builder builder = responsePrototype.newBuilderForType();
|
||||
|
@ -148,18 +144,15 @@ public final class CoprocessorRpcUtils {
|
|||
return response;
|
||||
}
|
||||
|
||||
public static org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.
|
||||
CoprocessorServiceResponse getResponse(final Message result, final byte [] regionName) {
|
||||
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.
|
||||
CoprocessorServiceResponse.Builder builder =
|
||||
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceResponse.
|
||||
newBuilder();
|
||||
builder.setRegion(RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME,
|
||||
regionName));
|
||||
public static ClientProtos.CoprocessorServiceResponse getResponse(final Message result,
|
||||
final byte[] regionName) {
|
||||
ClientProtos.CoprocessorServiceResponse.Builder builder =
|
||||
ClientProtos.CoprocessorServiceResponse.newBuilder();
|
||||
builder.setRegion(
|
||||
RequestConverter.buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName));
|
||||
// TODO: UGLY COPY IN HERE!!!!
|
||||
builder.setValue(builder.getValueBuilder().setName(result.getClass().getName())
|
||||
.setValue(org.apache.hbase.thirdparty.com.google.protobuf.ByteString.
|
||||
copyFrom(result.toByteArray())));
|
||||
builder.setValue(builder.getValueBuilder().setName(result.getClass().getName()).setValue(
|
||||
org.apache.hbase.thirdparty.com.google.protobuf.ByteString.copyFrom(result.toByteArray())));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
|
|
@ -18,13 +18,12 @@
|
|||
|
||||
package org.apache.hadoop.hbase.ipc;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
|
||||
/**
|
||||
* Used for server-side protobuf RPC service invocations. This handler allows
|
||||
|
|
|
@ -19,14 +19,14 @@ package org.apache.hadoop.hbase.security;
|
|||
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos.TokenIdentifier.Kind;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AuthenticationProtos.TokenIdentifier.Kind;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MasterService;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Maps RPC protocol interfaces to required configuration
|
||||
|
|
|
@ -33,11 +33,12 @@ import org.apache.hadoop.hbase.client.Table;
|
|||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.client.security.SecurityCapability;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService.BlockingInterface;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.AccessControlService.BlockingInterface;
|
||||
|
||||
/**
|
||||
* Utility client for doing access control admin operations.
|
||||
*/
|
||||
|
|
|
@ -22,24 +22,25 @@ import java.util.Collection;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.ArrayListMultimap;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.ListMultimap;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.AccessControlService;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse;
|
||||
|
||||
/**
|
||||
* @since 2.0.0
|
||||
|
@ -75,10 +76,10 @@ public class AccessControlUtil {
|
|||
permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
|
||||
|
||||
if (family != null) {
|
||||
permissionBuilder.setFamily(ByteStringer.wrap(family));
|
||||
permissionBuilder.setFamily(UnsafeByteOperations.unsafeWrap(family));
|
||||
}
|
||||
if (qualifier != null) {
|
||||
permissionBuilder.setQualifier(ByteStringer.wrap(qualifier));
|
||||
permissionBuilder.setQualifier(UnsafeByteOperations.unsafeWrap(qualifier));
|
||||
}
|
||||
ret.setType(AccessControlProtos.Permission.Type.Table)
|
||||
.setTablePermission(permissionBuilder);
|
||||
|
@ -336,10 +337,10 @@ public class AccessControlUtil {
|
|||
AccessControlProtos.TablePermission.newBuilder();
|
||||
builder.setTableName(ProtobufUtil.toProtoTableName(table.getTableName()));
|
||||
if (table.hasFamily()) {
|
||||
builder.setFamily(ByteStringer.wrap(table.getFamily()));
|
||||
builder.setFamily(UnsafeByteOperations.unsafeWrap(table.getFamily()));
|
||||
}
|
||||
if (table.hasQualifier()) {
|
||||
builder.setQualifier(ByteStringer.wrap(table.getQualifier()));
|
||||
builder.setQualifier(UnsafeByteOperations.unsafeWrap(table.getQualifier()));
|
||||
}
|
||||
Permission.Action[] actions = perm.getActions();
|
||||
if (actions != null) {
|
||||
|
@ -786,7 +787,7 @@ public class AccessControlUtil {
|
|||
AccessControlProtos.GetUserPermissionsRequest.Builder builder =
|
||||
AccessControlProtos.GetUserPermissionsRequest.newBuilder();
|
||||
if (namespace != null) {
|
||||
builder.setNamespaceName(ByteStringer.wrap(namespace));
|
||||
builder.setNamespaceName(UnsafeByteOperations.unsafeWrap(namespace));
|
||||
}
|
||||
if (!StringUtils.isEmpty(userName)) {
|
||||
builder.setUserName(ByteString.copyFromUtf8(userName));
|
||||
|
@ -827,12 +828,12 @@ public class AccessControlUtil {
|
|||
AccessControlProtos.TablePermission.Builder tablePermissionBuilder =
|
||||
AccessControlProtos.TablePermission.newBuilder();
|
||||
tablePermissionBuilder
|
||||
.setTableName(org.apache.hadoop.hbase.protobuf.ProtobufUtil.toProtoTableName(tableName));
|
||||
.setTableName(ProtobufUtil.toProtoTableName(tableName));
|
||||
if (Bytes.len(columnFamily) > 0) {
|
||||
tablePermissionBuilder.setFamily(ByteStringer.wrap(columnFamily));
|
||||
tablePermissionBuilder.setFamily(UnsafeByteOperations.unsafeWrap(columnFamily));
|
||||
}
|
||||
if (Bytes.len(columnQualifier) > 0) {
|
||||
tablePermissionBuilder.setQualifier(ByteString.copyFrom(columnQualifier));
|
||||
tablePermissionBuilder.setQualifier(UnsafeByteOperations.unsafeWrap(columnQualifier));
|
||||
}
|
||||
for (Permission.Action a : actions) {
|
||||
tablePermissionBuilder.addAction(toPermissionAction(a));
|
||||
|
@ -908,10 +909,10 @@ public class AccessControlUtil {
|
|||
permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
|
||||
}
|
||||
if (family != null) {
|
||||
permissionBuilder.setFamily(ByteStringer.wrap(family));
|
||||
permissionBuilder.setFamily(UnsafeByteOperations.unsafeWrap(family));
|
||||
}
|
||||
if (qualifier != null) {
|
||||
permissionBuilder.setQualifier(ByteStringer.wrap(qualifier));
|
||||
permissionBuilder.setQualifier(UnsafeByteOperations.unsafeWrap(qualifier));
|
||||
}
|
||||
ret.setType(AccessControlProtos.Permission.Type.Table)
|
||||
.setTablePermission(permissionBuilder);
|
||||
|
|
|
@ -21,15 +21,15 @@ package org.apache.hadoop.hbase.security.token;
|
|||
import java.io.DataInput;
|
||||
import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AuthenticationProtos;
|
||||
|
||||
/**
|
||||
* Represents the identity information stored in an HBase authentication token.
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.security.token;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.UndeclaredThrowableException;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
|
@ -31,7 +29,6 @@ import org.apache.hadoop.hbase.client.AsyncTable;
|
|||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
|
@ -39,7 +36,11 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AuthenticationProtos;
|
||||
|
||||
/**
|
||||
* Utility methods for obtaining authentication tokens, that do not require hbase-server.
|
||||
|
@ -69,7 +70,7 @@ public final class ClientTokenUtil {
|
|||
AsyncConnection conn) {
|
||||
CompletableFuture<Token<AuthenticationTokenIdentifier>> future = new CompletableFuture<>();
|
||||
if (injectedException != null) {
|
||||
future.completeExceptionally(injectedException);
|
||||
future.completeExceptionally(ProtobufUtil.handleRemoteException(injectedException));
|
||||
return future;
|
||||
}
|
||||
AsyncTable<?> table = conn.getTable(TableName.META_TABLE_NAME);
|
||||
|
|
|
@ -19,13 +19,9 @@ package org.apache.hadoop.hbase.security.visibility;
|
|||
|
||||
import static org.apache.hadoop.hbase.security.visibility.VisibilityConstants.LABELS_TABLE_NAME;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ServiceException;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
|
@ -33,19 +29,23 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
|||
import org.apache.hadoop.hbase.client.security.SecurityCapability;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.ListLabelsResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.GetAuthsRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.GetAuthsResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.ListLabelsRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.ListLabelsResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.SetAuthsRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabel;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
|
||||
|
||||
|
||||
/**
|
||||
* Utility client for doing visibility labels admin operations.
|
||||
|
@ -100,7 +100,7 @@ public class VisibilityClient {
|
|||
for (String label : labels) {
|
||||
if (label.length() > 0) {
|
||||
VisibilityLabel.Builder newBuilder = VisibilityLabel.newBuilder();
|
||||
newBuilder.setLabel(ByteStringer.wrap(Bytes.toBytes(label)));
|
||||
newBuilder.setLabel(UnsafeByteOperations.unsafeWrap((Bytes.toBytes(label))));
|
||||
builder.addVisLabel(newBuilder.build());
|
||||
}
|
||||
}
|
||||
|
@ -151,7 +151,7 @@ public class VisibilityClient {
|
|||
@Override
|
||||
public GetAuthsResponse call(VisibilityLabelsService service) throws IOException {
|
||||
GetAuthsRequest.Builder getAuthReqBuilder = GetAuthsRequest.newBuilder();
|
||||
getAuthReqBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(user)));
|
||||
getAuthReqBuilder.setUser(UnsafeByteOperations.unsafeWrap(Bytes.toBytes(user)));
|
||||
service.getAuths(controller, getAuthReqBuilder.build(), rpcCallback);
|
||||
GetAuthsResponse response = rpcCallback.get();
|
||||
if (controller.failedOnException()) {
|
||||
|
@ -235,7 +235,7 @@ public class VisibilityClient {
|
|||
@Override
|
||||
public VisibilityLabelsResponse call(VisibilityLabelsService service) throws IOException {
|
||||
SetAuthsRequest.Builder setAuthReqBuilder = SetAuthsRequest.newBuilder();
|
||||
setAuthReqBuilder.setUser(ByteStringer.wrap(Bytes.toBytes(user)));
|
||||
setAuthReqBuilder.setUser(UnsafeByteOperations.unsafeWrap(Bytes.toBytes(user)));
|
||||
for (String auth : auths) {
|
||||
if (auth.length() > 0) {
|
||||
setAuthReqBuilder.addAuth((ByteString.copyFromUtf8(auth)));
|
||||
|
|
|
@ -43,7 +43,6 @@ import java.util.concurrent.TimeUnit;
|
|||
import java.util.function.Function;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
|
||||
|
@ -424,6 +423,50 @@ public final class ProtobufUtil {
|
|||
return ServerName.valueOf(hostName, port, startCode);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a ServerName from the passed in data bytes.
|
||||
* @param data Data with a serialize server name in it; can handle the old style servername where
|
||||
* servername was host and port. Works too with data that begins w/ the pb 'PBUF' magic
|
||||
* and that is then followed by a protobuf that has a serialized {@link ServerName} in
|
||||
* it.
|
||||
* @return Returns null if <code>data</code> is null else converts passed data to a ServerName
|
||||
* instance.
|
||||
*/
|
||||
public static ServerName toServerName(final byte[] data) throws DeserializationException {
|
||||
if (data == null || data.length <= 0) {
|
||||
return null;
|
||||
}
|
||||
if (ProtobufMagic.isPBMagicPrefix(data)) {
|
||||
int prefixLen = ProtobufMagic.lengthOfPBMagic();
|
||||
try {
|
||||
ZooKeeperProtos.Master rss =
|
||||
ZooKeeperProtos.Master.parser().parseFrom(data, prefixLen, data.length - prefixLen);
|
||||
HBaseProtos.ServerName sn = rss.getMaster();
|
||||
return ServerName.valueOf(sn.getHostName(), sn.getPort(), sn.getStartCode());
|
||||
} catch (/* InvalidProtocolBufferException */IOException e) {
|
||||
// A failed parse of the znode is pretty catastrophic. Rather than loop
|
||||
// retrying hoping the bad bytes will changes, and rather than change
|
||||
// the signature on this method to add an IOE which will send ripples all
|
||||
// over the code base, throw a RuntimeException. This should "never" happen.
|
||||
// Fail fast if it does.
|
||||
throw new DeserializationException(e);
|
||||
}
|
||||
}
|
||||
// The str returned could be old style -- pre hbase-1502 -- which was
|
||||
// hostname and port seperated by a colon rather than hostname, port and
|
||||
// startcode delimited by a ','.
|
||||
String str = Bytes.toString(data);
|
||||
int index = str.indexOf(ServerName.SERVERNAME_SEPARATOR);
|
||||
if (index != -1) {
|
||||
// Presume its ServerName serialized with versioned bytes.
|
||||
return ServerName.parseVersionedServerName(data);
|
||||
}
|
||||
// Presume it a hostname:port format.
|
||||
String hostname = Addressing.parseHostname(str);
|
||||
int port = Addressing.parsePort(str);
|
||||
return ServerName.valueOf(hostname, port, -1L);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert a list of protocol buffer ServerName to a list of ServerName
|
||||
* @param proto protocol buffer ServerNameList
|
||||
|
|
|
@ -17,12 +17,12 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.shaded.protobuf;
|
||||
|
||||
import edu.umd.cs.findbugs.annotations.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
|
@ -31,14 +31,16 @@ import org.apache.hadoop.hbase.client.RegionInfo;
|
|||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.SingleResponse;
|
||||
import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.HasPermissionResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.HasPermissionResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CloseRegionResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetOnlineRegionResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetServerInfoResponse;
|
||||
|
@ -60,8 +62,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCatalog
|
|||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.RunCleanerChoreResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse;
|
||||
|
||||
import edu.umd.cs.findbugs.annotations.Nullable;
|
||||
|
||||
/**
|
||||
* Helper utility to build protocol buffer responses,
|
||||
* or retrieve data from protocol buffer responses.
|
||||
|
@ -341,7 +341,7 @@ public final class ResponseConverter {
|
|||
* @param controller the controller instance provided by the client when calling the service
|
||||
* @param ioe the exception encountered
|
||||
*/
|
||||
public static void setControllerException(com.google.protobuf.RpcController controller,
|
||||
public static void setControllerException(RpcController controller,
|
||||
IOException ioe) {
|
||||
if (controller != null) {
|
||||
if (controller instanceof ServerRpcController) {
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
package org.apache.hadoop.hbase.security.token;
|
||||
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertSame;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -38,6 +37,7 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
|
||||
|
@ -62,30 +62,29 @@ public class TestClientTokenUtil {
|
|||
Closeables.close(cl, true);
|
||||
}
|
||||
|
||||
private void assertException(Throwable injected, Throwable t) {
|
||||
while ((t = t.getCause()) != null) {
|
||||
if (t == injected) { // reference equality
|
||||
return;
|
||||
}
|
||||
}
|
||||
fail("can not find injected exception " + injected + ", actual exception is " + t);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testObtainToken() throws Exception {
|
||||
Throwable injected = new com.google.protobuf.ServiceException("injected");
|
||||
Exception injected = new Exception("injected");
|
||||
|
||||
Class<?> clientTokenUtil = cl.loadClass(ClientTokenUtil.class.getCanonicalName());
|
||||
Field shouldInjectFault = clientTokenUtil.getDeclaredField("injectedException");
|
||||
shouldInjectFault.setAccessible(true);
|
||||
shouldInjectFault.set(null, injected);
|
||||
shouldInjectFault.set(null, new ServiceException(injected));
|
||||
|
||||
try {
|
||||
ClientTokenUtil.obtainToken((Connection)null);
|
||||
fail("Should have injected exception.");
|
||||
} catch (IOException e) {
|
||||
Throwable t = e;
|
||||
boolean serviceExceptionFound = false;
|
||||
while ((t = t.getCause()) != null) {
|
||||
if (t == injected) { // reference equality
|
||||
serviceExceptionFound = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
if (!serviceExceptionFound) {
|
||||
throw e; // wrong exception, fail the test
|
||||
}
|
||||
assertException(injected, e);
|
||||
}
|
||||
|
||||
CompletableFuture<?> future = ClientTokenUtil.obtainToken((AsyncConnection)null);
|
||||
|
@ -93,7 +92,7 @@ public class TestClientTokenUtil {
|
|||
future.get();
|
||||
fail("Should have injected exception.");
|
||||
} catch (ExecutionException e) {
|
||||
assertSame(injected, e.getCause());
|
||||
assertException(injected, e);
|
||||
}
|
||||
Boolean loaded = (Boolean) cl.loadClass(ProtobufUtil.class.getCanonicalName())
|
||||
.getDeclaredMethod("isClassLoaderLoaded").invoke(null);
|
||||
|
|
|
@ -47,24 +47,6 @@
|
|||
<skipAssembly>true</skipAssembly>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.xolstice.maven.plugins</groupId>
|
||||
<artifactId>protobuf-maven-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>compile-protoc</id>
|
||||
<phase>generate-sources</phase>
|
||||
<goals>
|
||||
<goal>compile</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<additionalProtoPathElements>
|
||||
<additionalProtoPathElement>${basedir}/../hbase-protocol/src/main/protobuf</additionalProtoPathElement>
|
||||
</additionalProtoPathElements>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>net.revelc.code</groupId>
|
||||
<artifactId>warbucks-maven-plugin</artifactId>
|
||||
|
|
|
@ -20,11 +20,6 @@ package org.apache.hadoop.hbase.client.coprocessor;
|
|||
import static org.apache.hadoop.hbase.client.coprocessor.AggregationHelper.getParsedGenericInstance;
|
||||
import static org.apache.hadoop.hbase.client.coprocessor.AggregationHelper.validateArgAndGetPB;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -35,7 +30,6 @@ import java.util.NavigableMap;
|
|||
import java.util.NavigableSet;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
|
@ -48,15 +42,21 @@ import org.apache.hadoop.hbase.client.Scan;
|
|||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateService;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AggregateProtos.AggregateRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AggregateProtos.AggregateResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AggregateProtos.AggregateService;
|
||||
|
||||
/**
|
||||
* This client class is for invoking the aggregate functions deployed on the
|
||||
* Region Server side via the AggregateService. This class will implement the
|
||||
|
|
|
@ -17,23 +17,24 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.client.coprocessor;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.Message;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.lang.reflect.ParameterizedType;
|
||||
import java.lang.reflect.Type;
|
||||
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateRequest;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AggregateProtos.AggregateRequest;
|
||||
|
||||
/**
|
||||
* Helper class for constructing aggregation request and response.
|
||||
*/
|
||||
|
|
|
@ -21,7 +21,6 @@ import static org.apache.hadoop.hbase.client.coprocessor.AggregationHelper.getPa
|
|||
import static org.apache.hadoop.hbase.client.coprocessor.AggregationHelper.validateArgAndGetPB;
|
||||
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
|
||||
|
||||
import com.google.protobuf.Message;
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.NavigableMap;
|
||||
|
@ -38,13 +37,16 @@ import org.apache.hadoop.hbase.client.RegionInfo;
|
|||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.coprocessor.ColumnInterpreter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateService;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AggregateProtos.AggregateRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AggregateProtos.AggregateResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AggregateProtos.AggregateService;
|
||||
|
||||
/**
|
||||
* This client class is for invoking the aggregate functions deployed on the Region Server side via
|
||||
* the AggregateService. This class will implement the supporting functionality for
|
||||
|
|
|
@ -19,12 +19,6 @@ package org.apache.hadoop.hbase.coprocessor;
|
|||
|
||||
import static org.apache.hadoop.hbase.client.coprocessor.AggregationHelper.getParsedGenericInstance;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.nio.ByteBuffer;
|
||||
|
@ -32,21 +26,27 @@ import java.util.ArrayList;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.NavigableSet;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AggregateProtos.AggregateService;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AggregateProtos.AggregateRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AggregateProtos.AggregateResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AggregateProtos.AggregateService;
|
||||
|
||||
/**
|
||||
* A concrete AggregateProtocol implementation. Its system level coprocessor
|
||||
* that computes the aggregate function at a region level.
|
||||
|
|
|
@ -18,10 +18,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
|
@ -31,7 +27,6 @@ import java.util.LinkedList;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.commons.lang3.ArrayUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
@ -54,9 +49,6 @@ import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
|||
import org.apache.hadoop.hbase.mapreduce.ExportUtils;
|
||||
import org.apache.hadoop.hbase.mapreduce.Import;
|
||||
import org.apache.hadoop.hbase.mapreduce.ResultSerialization;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.DelegationToken;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ExportProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
|
@ -64,7 +56,6 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
|||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.security.UserProvider;
|
||||
import org.apache.hadoop.hbase.security.token.FsDelegationToken;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Triple;
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
|
@ -79,7 +70,16 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.DelegationToken;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ExportProtos;
|
||||
|
||||
/**
|
||||
* Export an HBase table. Writes content to sequence files up in HDFS. Use
|
||||
|
@ -287,8 +287,8 @@ public class Export extends ExportProtos.ExportService implements RegionCoproces
|
|||
DelegationToken protoToken = null;
|
||||
if (userToken != null) {
|
||||
protoToken = DelegationToken.newBuilder()
|
||||
.setIdentifier(ByteStringer.wrap(userToken.getIdentifier()))
|
||||
.setPassword(ByteStringer.wrap(userToken.getPassword()))
|
||||
.setIdentifier(UnsafeByteOperations.unsafeWrap(userToken.getIdentifier()))
|
||||
.setPassword(UnsafeByteOperations.unsafeWrap(userToken.getPassword()))
|
||||
.setKind(userToken.getKind().toString())
|
||||
.setService(userToken.getService().toString()).build();
|
||||
}
|
||||
|
|
|
@ -17,28 +17,28 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.ColumnAggregationService;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationProtos.ColumnAggregationService;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationProtos.SumRequest;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse;
|
||||
|
||||
/**
|
||||
* The aggregation implementation at a region.
|
||||
*/
|
||||
|
|
|
@ -17,23 +17,15 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.ColumnAggregationNullResponseSumRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.ColumnAggregationNullResponseSumResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.ColumnAggregationServiceNullResponse;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
|
@ -41,6 +33,14 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.ColumnAggregationNullResponseSumRequest;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.ColumnAggregationNullResponseSumResponse;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.ColumnAggregationServiceNullResponse;
|
||||
|
||||
/**
|
||||
* Test coprocessor endpoint that always returns {@code null} for requests to the last region
|
||||
* in the table. This allows tests to provide assurance of correct {@code null} handling for
|
||||
|
|
|
@ -17,24 +17,16 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.ColumnAggregationWithErrorsSumRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.ColumnAggregationWithErrorsSumResponse;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
|
@ -42,6 +34,14 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.ColumnAggregationWithErrorsSumRequest;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.ColumnAggregationWithErrorsSumResponse;
|
||||
|
||||
/**
|
||||
* Test coprocessor endpoint that always throws a {@link DoNotRetryIOException} for requests on
|
||||
* the last region in the table. This allows tests to ensure correct error handling of
|
||||
|
|
|
@ -17,24 +17,25 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos;
|
||||
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.AddrResponseProto;
|
||||
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyRequestProto;
|
||||
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.EmptyResponseProto;
|
||||
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos.PauseRequestProto;
|
||||
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos;
|
||||
import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.AddrResponseProto;
|
||||
import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyRequestProto;
|
||||
import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.EmptyResponseProto;
|
||||
import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos.PauseRequestProto;
|
||||
import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos;
|
||||
|
||||
/**
|
||||
* Test implementation of a coprocessor endpoint exposing the
|
||||
* {@link org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos.TestProtobufRpcProto}
|
||||
|
|
|
@ -21,9 +21,6 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
|
@ -34,13 +31,7 @@ import org.apache.hadoop.hbase.ServerName;
|
|||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
|
||||
import org.apache.hadoop.hbase.client.TestAsyncAdminBase;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.DummyRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.DummyResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.DummyService;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos;
|
||||
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos;
|
||||
import org.apache.hadoop.hbase.testclassification.ClientTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.junit.BeforeClass;
|
||||
|
@ -50,6 +41,17 @@ import org.junit.experimental.categories.Category;
|
|||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.DummyRequest;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.DummyResponse;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.DummyService;
|
||||
import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos;
|
||||
import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
@Category({ ClientTests.class, MediumTests.class })
|
||||
public class TestAsyncCoprocessorEndpoint extends TestAsyncAdminBase {
|
||||
|
|
|
@ -20,8 +20,6 @@ package org.apache.hadoop.hbase.coprocessor;
|
|||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
@ -35,14 +33,6 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.ColumnAggregationWithErrorsSumRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.ColumnAggregationWithErrorsSumResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.ColumnAggregationNullResponseSumRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.ColumnAggregationNullResponseSumResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.ColumnAggregationServiceNullResponse;
|
||||
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -54,6 +44,18 @@ import org.junit.experimental.categories.Category;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationProtos;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.ColumnAggregationWithErrorsSumRequest;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.ColumnAggregationWithErrorsSumResponse;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.ColumnAggregationNullResponseSumRequest;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.ColumnAggregationNullResponseSumResponse;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.ColumnAggregationServiceNullResponse;
|
||||
|
||||
/**
|
||||
* TestEndpoint: test cases to verify the batch execution of coprocessor Endpoint
|
||||
*/
|
||||
|
|
|
@ -23,8 +23,6 @@ import static org.junit.Assert.assertNull;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -44,15 +42,11 @@ import org.apache.hadoop.hbase.client.RegionLocator;
|
|||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos;
|
||||
import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos;
|
||||
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
|
@ -62,6 +56,14 @@ import org.junit.experimental.categories.Category;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationProtos;
|
||||
import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestProtos;
|
||||
import org.apache.hadoop.hbase.shaded.ipc.protobuf.generated.TestRpcServiceProtos;
|
||||
|
||||
/**
|
||||
* TestEndpoint: test cases to verify coprocessor Endpoint
|
||||
*/
|
||||
|
@ -133,9 +135,9 @@ public class TestCoprocessorEndpoint {
|
|||
new CoprocessorRpcUtils.BlockingRpcCallback<>();
|
||||
ColumnAggregationProtos.SumRequest.Builder builder =
|
||||
ColumnAggregationProtos.SumRequest.newBuilder();
|
||||
builder.setFamily(ByteStringer.wrap(family));
|
||||
builder.setFamily(UnsafeByteOperations.unsafeWrap(family));
|
||||
if (qualifier != null && qualifier.length > 0) {
|
||||
builder.setQualifier(ByteStringer.wrap(qualifier));
|
||||
builder.setQualifier(UnsafeByteOperations.unsafeWrap(qualifier));
|
||||
}
|
||||
instance.sum(null, builder.build(), rpcCallback);
|
||||
return rpcCallback.get().getSum();
|
||||
|
|
|
@ -1,128 +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.coprocessor;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.DummyRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.DummyResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.DummyService;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* Tests to ensure that 2.0 is backward compatible in loading CoprocessorService.
|
||||
*/
|
||||
@Category({MediumTests.class})
|
||||
public class TestCoprocessorServiceBackwardCompatibility {
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestCoprocessorServiceBackwardCompatibility.class);
|
||||
|
||||
private static HBaseTestingUtility TEST_UTIL = null;
|
||||
private static Configuration CONF = null;
|
||||
|
||||
private static final long MASTER = 1;
|
||||
private static final long REGIONSERVER = 2;
|
||||
private static final long REGION = 3;
|
||||
|
||||
public static class DummyCoprocessorService extends DummyService
|
||||
implements CoprocessorService, SingletonCoprocessorService {
|
||||
// depending on the value passed thru DummyRequest, the following fields would be incremented
|
||||
// value == MASTER
|
||||
static int numMaster = 0;
|
||||
// value == REGIONSERVER
|
||||
static int numRegionServer = 0;
|
||||
// value == REGION
|
||||
static int numRegion = 0;
|
||||
|
||||
@Override
|
||||
public Service getService() {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void dummyCall(RpcController controller, DummyRequest request,
|
||||
RpcCallback<DummyResponse> callback) {
|
||||
callback.run(DummyResponse.newBuilder().setValue("").build());
|
||||
if (request.getValue() == MASTER) {
|
||||
numMaster += request.getValue();
|
||||
} else if (request.getValue() == REGIONSERVER) {
|
||||
numRegionServer += request.getValue();
|
||||
} else if (request.getValue() == REGION) {
|
||||
numRegion += request.getValue();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void dummyThrow(RpcController controller, DummyRequest request,
|
||||
RpcCallback<DummyResponse> callback) {
|
||||
}
|
||||
}
|
||||
|
||||
@BeforeClass
|
||||
public static void setupBeforeClass() throws Exception {
|
||||
TEST_UTIL = new HBaseTestingUtility();
|
||||
CONF = TEST_UTIL.getConfiguration();
|
||||
CONF.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
|
||||
DummyCoprocessorService.class.getName());
|
||||
CONF.setStrings(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY,
|
||||
DummyCoprocessorService.class.getName());
|
||||
CONF.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
|
||||
DummyCoprocessorService.class.getName());
|
||||
TEST_UTIL.startMiniCluster();
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDownAfter() throws Exception {
|
||||
TEST_UTIL.shutdownMiniCluster();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCoprocessorServiceLoadedByMaster() throws Throwable {
|
||||
TEST_UTIL.getAdmin().coprocessorService().callBlockingMethod(
|
||||
DummyCoprocessorService.getDescriptor().findMethodByName("dummyCall"), null,
|
||||
DummyRequest.newBuilder().setValue(MASTER).build(), DummyResponse.getDefaultInstance());
|
||||
assertEquals(MASTER, DummyCoprocessorService.numMaster);
|
||||
|
||||
TEST_UTIL.getAdmin().coprocessorService(
|
||||
TEST_UTIL.getHBaseCluster().getRegionServer(0).getServerName()).callBlockingMethod(
|
||||
DummyCoprocessorService.getDescriptor().findMethodByName("dummyCall"), null,
|
||||
DummyRequest.newBuilder().setValue(REGIONSERVER).build(),
|
||||
DummyResponse.getDefaultInstance());
|
||||
assertEquals(REGIONSERVER, DummyCoprocessorService.numRegionServer);
|
||||
|
||||
TEST_UTIL.getConnection().getTable(TableName.valueOf("hbase:meta")).batchCoprocessorService(
|
||||
DummyCoprocessorService.getDescriptor().findMethodByName("dummyCall"),
|
||||
DummyRequest.newBuilder().setValue(REGION).build(), Bytes.toBytes(""), Bytes.toBytes(""),
|
||||
DummyResponse.getDefaultInstance());
|
||||
assertEquals(REGION, DummyCoprocessorService.numRegion);
|
||||
}
|
||||
}
|
|
@ -19,8 +19,6 @@ package org.apache.hadoop.hbase.coprocessor;
|
|||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
|
@ -33,7 +31,6 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
|
@ -46,6 +43,11 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.ColumnAggregationProtos;
|
||||
|
||||
@Category({CoprocessorTests.class, MediumTests.class})
|
||||
public class TestCoprocessorTableEndpoint {
|
||||
@ClassRule
|
||||
|
|
|
@ -20,22 +20,14 @@ package org.apache.hadoop.hbase.coprocessor;
|
|||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.util.Collections;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.DummyRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.DummyResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.DummyService;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.junit.AfterClass;
|
||||
|
@ -44,6 +36,16 @@ import org.junit.ClassRule;
|
|||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.DummyRequest;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.DummyResponse;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos.DummyService;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
|
||||
@Category({CoprocessorTests.class, MediumTests.class})
|
||||
public class TestRegionServerCoprocessorEndpoint {
|
||||
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.coprocessor;
|
|||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.google.protobuf.Message;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
|
@ -48,26 +47,13 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.coprocessor.RowProcessorClient;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.FriendsOfFriendsProcessorRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.FriendsOfFriendsProcessorResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.IncCounterProcessorRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.IncCounterProcessorResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.RowSwapProcessorRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.RowSwapProcessorResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.TimeoutProcessorRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.TimeoutProcessorResponse;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.ipc.RpcScheduler;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.RowProcessorService;
|
||||
import org.apache.hadoop.hbase.regionserver.BaseRowProcessor;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.ByteStringer;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.junit.AfterClass;
|
||||
|
@ -78,6 +64,22 @@ import org.junit.experimental.categories.Category;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.FriendsOfFriendsProcessorRequest;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.FriendsOfFriendsProcessorResponse;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.IncCounterProcessorRequest;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.IncCounterProcessorResponse;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.RowSwapProcessorRequest;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.RowSwapProcessorResponse;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.TimeoutProcessorRequest;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.TimeoutProcessorResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RowProcessorProtos.ProcessRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RowProcessorProtos.ProcessResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RowProcessorProtos.RowProcessorService;
|
||||
|
||||
/**
|
||||
* Verifies ProcessEndpoint works.
|
||||
* The tested RowProcessor performs two scans and a read-modify-write.
|
||||
|
@ -383,7 +385,7 @@ public class TestRowProcessorEndpoint {
|
|||
public IncCounterProcessorRequest getRequestData() throws IOException {
|
||||
IncCounterProcessorRequest.Builder builder = IncCounterProcessorRequest.newBuilder();
|
||||
builder.setCounter(counter);
|
||||
builder.setRow(ByteStringer.wrap(row));
|
||||
builder.setRow(UnsafeByteOperations.unsafeWrap(row));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -462,8 +464,8 @@ public class TestRowProcessorEndpoint {
|
|||
public FriendsOfFriendsProcessorRequest getRequestData() throws IOException {
|
||||
FriendsOfFriendsProcessorRequest.Builder builder =
|
||||
FriendsOfFriendsProcessorRequest.newBuilder();
|
||||
builder.setPerson(ByteStringer.wrap(person));
|
||||
builder.setRow(ByteStringer.wrap(row));
|
||||
builder.setPerson(UnsafeByteOperations.unsafeWrap(person));
|
||||
builder.setRow(UnsafeByteOperations.unsafeWrap(row));
|
||||
builder.addAllResult(result);
|
||||
FriendsOfFriendsProcessorRequest f = builder.build();
|
||||
return f;
|
||||
|
@ -571,8 +573,8 @@ public class TestRowProcessorEndpoint {
|
|||
@Override
|
||||
public RowSwapProcessorRequest getRequestData() throws IOException {
|
||||
RowSwapProcessorRequest.Builder builder = RowSwapProcessorRequest.newBuilder();
|
||||
builder.setRow1(ByteStringer.wrap(row1));
|
||||
builder.setRow2(ByteStringer.wrap(row2));
|
||||
builder.setRow1(UnsafeByteOperations.unsafeWrap(row1));
|
||||
builder.setRow2(UnsafeByteOperations.unsafeWrap(row2));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
@ -630,7 +632,7 @@ public class TestRowProcessorEndpoint {
|
|||
@Override
|
||||
public TimeoutProcessorRequest getRequestData() throws IOException {
|
||||
TimeoutProcessorRequest.Builder builder = TimeoutProcessorRequest.newBuilder();
|
||||
builder.setRow(ByteStringer.wrap(row));
|
||||
builder.setRow(UnsafeByteOperations.unsafeWrap(row));
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.coprocessor;
|
|||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import com.google.protobuf.ServiceException;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
|
@ -81,6 +80,8 @@ import org.junit.rules.TestName;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
|
||||
@Category({MediumTests.class})
|
||||
public class TestSecureExport {
|
||||
@ClassRule
|
||||
|
|
|
@ -19,15 +19,17 @@ package org.apache.hadoop.hbase.ipc;
|
|||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import com.google.protobuf.Descriptors;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AuthenticationProtos;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.ServiceDescriptor;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.DummyRegionServerEndpointProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AuthenticationProtos;
|
||||
|
||||
@Category(SmallTests.class)
|
||||
public class TestCoprocessorRpcUtils {
|
||||
|
||||
|
@ -38,12 +40,12 @@ public class TestCoprocessorRpcUtils {
|
|||
@Test
|
||||
public void testServiceName() throws Exception {
|
||||
// verify that we de-namespace build in HBase rpc services
|
||||
Descriptors.ServiceDescriptor authService =
|
||||
ServiceDescriptor authService =
|
||||
AuthenticationProtos.AuthenticationService.getDescriptor();
|
||||
assertEquals(authService.getName(), CoprocessorRpcUtils.getServiceName(authService));
|
||||
|
||||
// non-hbase rpc services should remain fully qualified
|
||||
Descriptors.ServiceDescriptor dummyService =
|
||||
ServiceDescriptor dummyService =
|
||||
DummyRegionServerEndpointProtos.DummyService.getDescriptor();
|
||||
assertEquals(dummyService.getFullName(), CoprocessorRpcUtils.getServiceName(dummyService));
|
||||
}
|
||||
|
|
|
@ -21,10 +21,6 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
|
@ -41,17 +37,6 @@ import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
|
|||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.CountRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.CountResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.HelloRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.HelloResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.IncrementCountRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.IncrementCountResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.NoopRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.NoopResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.PingRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.protobuf.generated.PingProtos.PingResponse;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
|
@ -66,6 +51,23 @@ import org.junit.experimental.categories.Category;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos.CountRequest;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos.CountResponse;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos.HelloRequest;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos.HelloResponse;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos.IncrementCountRequest;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos.IncrementCountResponse;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos.NoopRequest;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos.NoopResponse;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos.PingRequest;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated.PingProtos.PingResponse;
|
||||
|
||||
@Category({RegionServerTests.class, MediumTests.class})
|
||||
public class TestServerCustomProtocol {
|
||||
@ClassRule
|
||||
|
|
|
@ -31,13 +31,14 @@ import org.apache.hadoop.hbase.client.Table;
|
|||
import org.apache.hadoop.hbase.client.coprocessor.Batch;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils.BlockingRpcCallback;
|
||||
import org.apache.hadoop.hbase.ipc.ServerRpcController;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RefreshHFilesProtos;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RefreshHFilesProtos;
|
||||
|
||||
/**
|
||||
* This client class is for invoking the refresh HFile function deployed on the
|
||||
* Region Server side via the RefreshHFilesService.
|
||||
|
|
|
@ -17,9 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.coprocessor.example;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
|
@ -38,14 +35,8 @@ import org.apache.hadoop.hbase.client.Scan;
|
|||
import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest;
|
||||
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType;
|
||||
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse;
|
||||
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.Builder;
|
||||
import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteService;
|
||||
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.OperationStatus;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
|
@ -54,6 +45,17 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.Builder;
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteService;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
|
||||
/**
|
||||
* Defines a protocol to delete data in bulk based on a scan. The scan can be range scan or with
|
||||
* conditions(filters) etc.This can be used to delete rows, column family(s), column qualifier(s)
|
||||
|
|
|
@ -17,9 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.coprocessor.example;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
|
@ -27,12 +24,17 @@ import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
|
|||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RefreshHFilesProtos;
|
||||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RefreshHFilesProtos;
|
||||
|
||||
/**
|
||||
* Coprocessor endpoint to refresh HFiles on replica.
|
||||
* <p>
|
||||
|
|
|
@ -18,9 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.coprocessor.example;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
|
@ -32,13 +29,18 @@ import org.apache.hadoop.hbase.client.Scan;
|
|||
import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos;
|
||||
import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.InternalScanner;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.coprocessor.example.generated.ExampleProtos;
|
||||
|
||||
/**
|
||||
* Sample coprocessor endpoint exposing a Service interface for counting rows and key values.
|
||||
*
|
||||
|
|
|
@ -18,10 +18,8 @@
|
|||
package org.apache.hadoop.hbase;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.client.TestMetaWithReplicas;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
|
||||
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
|
@ -31,8 +29,8 @@ import org.junit.AfterClass;
|
|||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
|
||||
/**
|
||||
* An integration test that starts the cluster with three replicas for the meta
|
||||
|
@ -43,7 +41,7 @@ import org.slf4j.LoggerFactory;
|
|||
*/
|
||||
@Category(IntegrationTests.class)
|
||||
public class IntegrationTestMetaReplicas {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(IntegrationTestMetaReplicas.class);
|
||||
|
||||
/**
|
||||
* Util to get at the cluster.
|
||||
*/
|
||||
|
|
|
@ -128,6 +128,15 @@ message HasUserPermissionsResponse {
|
|||
repeated bool has_user_permission = 1;
|
||||
}
|
||||
|
||||
message HasPermissionRequest {
|
||||
required TablePermission table_permission = 1;
|
||||
required bytes user_name = 2;
|
||||
}
|
||||
|
||||
message HasPermissionResponse {
|
||||
optional bool has_permission = 1;
|
||||
}
|
||||
|
||||
service AccessControlService {
|
||||
rpc Grant(GrantRequest)
|
||||
returns (GrantResponse);
|
||||
|
@ -140,4 +149,7 @@ service AccessControlService {
|
|||
|
||||
rpc CheckPermissions(CheckPermissionsRequest)
|
||||
returns (CheckPermissionsResponse);
|
||||
|
||||
rpc HasPermission(HasPermissionRequest)
|
||||
returns (HasPermissionResponse);
|
||||
}
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
syntax = "proto2";
|
||||
package hbase.pb;
|
||||
|
||||
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
|
||||
option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";
|
||||
option java_outer_classname = "AggregateProtos";
|
||||
option java_generic_services = true;
|
||||
option java_generate_equals_and_hash = true;
|
|
@ -0,0 +1,83 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
syntax = "proto2";
|
||||
package hbase.pb;
|
||||
|
||||
option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";
|
||||
option java_outer_classname = "AuthenticationProtos";
|
||||
option java_generic_services = true;
|
||||
option java_generate_equals_and_hash = true;
|
||||
option optimize_for = SPEED;
|
||||
|
||||
message AuthenticationKey {
|
||||
required int32 id = 1;
|
||||
required int64 expiration_date = 2;
|
||||
required bytes key = 3;
|
||||
}
|
||||
|
||||
|
||||
message TokenIdentifier {
|
||||
enum Kind {
|
||||
HBASE_AUTH_TOKEN = 0;
|
||||
}
|
||||
required Kind kind = 1;
|
||||
required bytes username = 2;
|
||||
required int32 key_id = 3;
|
||||
optional int64 issue_date = 4;
|
||||
optional int64 expiration_date = 5;
|
||||
optional int64 sequence_number = 6;
|
||||
}
|
||||
|
||||
|
||||
// Serialization of the org.apache.hadoop.security.token.Token class
|
||||
// Note that this is a Hadoop class, so fields may change!
|
||||
message Token {
|
||||
// the TokenIdentifier in serialized form
|
||||
// Note: we can't use the protobuf directly because the Hadoop Token class
|
||||
// only stores the serialized bytes
|
||||
optional bytes identifier = 1;
|
||||
optional bytes password = 2;
|
||||
optional bytes service = 3;
|
||||
}
|
||||
|
||||
|
||||
// RPC request & response messages
|
||||
message GetAuthenticationTokenRequest {
|
||||
}
|
||||
|
||||
message GetAuthenticationTokenResponse {
|
||||
optional Token token = 1;
|
||||
}
|
||||
|
||||
message WhoAmIRequest {
|
||||
}
|
||||
|
||||
message WhoAmIResponse {
|
||||
optional string username = 1;
|
||||
optional string auth_method = 2;
|
||||
}
|
||||
|
||||
|
||||
// RPC service
|
||||
service AuthenticationService {
|
||||
rpc GetAuthenticationToken(GetAuthenticationTokenRequest)
|
||||
returns (GetAuthenticationTokenResponse);
|
||||
|
||||
rpc WhoAmI(WhoAmIRequest)
|
||||
returns (WhoAmIResponse);
|
||||
}
|
|
@ -18,7 +18,7 @@
|
|||
syntax = "proto2";
|
||||
package hbase.pb;
|
||||
|
||||
option java_package = "org.apache.hadoop.hbase.coprocessor.example.generated";
|
||||
option java_package = "org.apache.hadoop.hbase.shaded.coprocessor.example.generated";
|
||||
option java_outer_classname = "BulkDeleteProtos";
|
||||
option java_generic_services = true;
|
||||
option java_generate_equals_and_hash = true;
|
|
@ -18,7 +18,7 @@
|
|||
syntax = "proto2";
|
||||
|
||||
// Coprocessor test
|
||||
option java_package = "org.apache.hadoop.hbase.coprocessor.protobuf.generated";
|
||||
option java_package = "org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated";
|
||||
option java_outer_classname = "ColumnAggregationWithNullResponseProtos";
|
||||
option java_generic_services = true;
|
||||
option java_generate_equals_and_hash = true;
|
|
@ -18,7 +18,7 @@
|
|||
syntax = "proto2";
|
||||
|
||||
// Coprocessor test
|
||||
option java_package = "org.apache.hadoop.hbase.coprocessor.protobuf.generated";
|
||||
option java_package = "org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated";
|
||||
option java_outer_classname = "ColumnAggregationProtos";
|
||||
option java_generic_services = true;
|
||||
option java_generate_equals_and_hash = true;
|
|
@ -18,7 +18,7 @@
|
|||
syntax = "proto2";
|
||||
|
||||
// Coprocessor test
|
||||
option java_package = "org.apache.hadoop.hbase.coprocessor.protobuf.generated";
|
||||
option java_package = "org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated";
|
||||
option java_outer_classname = "ColumnAggregationWithErrorsProtos";
|
||||
option java_generic_services = true;
|
||||
option java_generate_equals_and_hash = true;
|
|
@ -19,7 +19,7 @@ syntax = "proto2";
|
|||
package hbase.test.pb;
|
||||
|
||||
// Coprocessor test
|
||||
option java_package = "org.apache.hadoop.hbase.coprocessor.protobuf.generated";
|
||||
option java_package = "org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated";
|
||||
option java_outer_classname = "DummyRegionServerEndpointProtos";
|
||||
option java_generic_services = true;
|
||||
option java_generate_equals_and_hash = true;
|
|
@ -18,7 +18,7 @@
|
|||
syntax = "proto2";
|
||||
package hbase.pb;
|
||||
|
||||
option java_package = "org.apache.hadoop.hbase.coprocessor.example.generated";
|
||||
option java_package = "org.apache.hadoop.hbase.shaded.coprocessor.example.generated";
|
||||
option java_outer_classname = "ExampleProtos";
|
||||
option java_generic_services = true;
|
||||
option java_generate_equals_and_hash = true;
|
|
@ -18,7 +18,7 @@
|
|||
syntax = "proto2";
|
||||
package hbase.pb;
|
||||
|
||||
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
|
||||
option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";
|
||||
option java_outer_classname = "ExportProtos";
|
||||
option java_generate_equals_and_hash = true;
|
||||
option optimize_for = SPEED;
|
|
@ -17,7 +17,7 @@
|
|||
*/
|
||||
syntax = "proto2";
|
||||
|
||||
option java_package = "org.apache.hadoop.hbase.coprocessor.protobuf.generated";
|
||||
option java_package = "org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated";
|
||||
option java_outer_classname = "IncrementCounterProcessorTestProtos";
|
||||
option java_generate_equals_and_hash = true;
|
||||
|
|
@ -0,0 +1,48 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
syntax = "proto2";
|
||||
package hbase.pb;
|
||||
|
||||
import "Client.proto";
|
||||
import "HBase.proto";
|
||||
option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";
|
||||
option java_outer_classname = "MultiRowMutationProtos";
|
||||
option java_generate_equals_and_hash = true;
|
||||
option java_generic_services = true;
|
||||
option optimize_for = SPEED;
|
||||
|
||||
message MultiRowMutationProcessorRequest{
|
||||
}
|
||||
|
||||
message MultiRowMutationProcessorResponse{
|
||||
}
|
||||
|
||||
message MutateRowsRequest {
|
||||
repeated MutationProto mutation_request = 1;
|
||||
optional uint64 nonce_group = 2;
|
||||
optional uint64 nonce = 3;
|
||||
optional RegionSpecifier region = 4;
|
||||
}
|
||||
|
||||
message MutateRowsResponse {
|
||||
}
|
||||
|
||||
service MultiRowMutationService {
|
||||
rpc MutateRows(MutateRowsRequest)
|
||||
returns(MutateRowsResponse);
|
||||
}
|
|
@ -0,0 +1,67 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
// Coprocessor test
|
||||
syntax = "proto2";
|
||||
option java_package = "org.apache.hadoop.hbase.shaded.coprocessor.protobuf.generated";
|
||||
option java_outer_classname = "PingProtos";
|
||||
option java_generic_services = true;
|
||||
option java_generate_equals_and_hash = true;
|
||||
|
||||
message PingRequest {
|
||||
}
|
||||
|
||||
message PingResponse {
|
||||
required string pong = 1;
|
||||
}
|
||||
|
||||
message CountRequest {
|
||||
}
|
||||
|
||||
message CountResponse {
|
||||
required int32 count = 1;
|
||||
}
|
||||
|
||||
message IncrementCountRequest {
|
||||
required int32 diff = 1;
|
||||
}
|
||||
|
||||
message IncrementCountResponse {
|
||||
required int32 count = 1;
|
||||
}
|
||||
|
||||
message HelloRequest {
|
||||
optional string name = 1;
|
||||
}
|
||||
|
||||
message HelloResponse {
|
||||
optional string response = 1;
|
||||
}
|
||||
|
||||
message NoopRequest {
|
||||
}
|
||||
|
||||
message NoopResponse {
|
||||
}
|
||||
|
||||
service PingService {
|
||||
rpc ping(PingRequest) returns(PingResponse);
|
||||
rpc count(CountRequest) returns(CountResponse);
|
||||
rpc increment(IncrementCountRequest) returns(IncrementCountResponse);
|
||||
rpc hello(HelloRequest) returns(HelloResponse);
|
||||
rpc noop(NoopRequest) returns(NoopResponse);
|
||||
}
|
|
@ -18,7 +18,7 @@
|
|||
syntax = "proto2";
|
||||
package hbase.pb;
|
||||
|
||||
option java_package = "org.apache.hadoop.hbase.protobuf.generated";
|
||||
option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";
|
||||
option java_outer_classname = "RefreshHFilesProtos";
|
||||
option java_generic_services = true;
|
||||
option java_generate_equals_and_hash = true;
|
|
@ -0,0 +1,46 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
/**
|
||||
* Defines a protocol to perform multi row transactions.
|
||||
* See BaseRowProcessorEndpoint for the implementation.
|
||||
* See HRegion#processRowsWithLocks() for details.
|
||||
*/
|
||||
syntax = "proto2";
|
||||
package hbase.pb;
|
||||
|
||||
option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";
|
||||
option java_outer_classname = "RowProcessorProtos";
|
||||
option java_generic_services = true;
|
||||
option java_generate_equals_and_hash = true;
|
||||
option optimize_for = SPEED;
|
||||
|
||||
message ProcessRequest {
|
||||
required string row_processor_class_name = 1;
|
||||
optional string row_processor_initializer_message_name = 2;
|
||||
optional bytes row_processor_initializer_message = 3;
|
||||
optional uint64 nonce_group = 4;
|
||||
optional uint64 nonce = 5;
|
||||
}
|
||||
|
||||
message ProcessResponse {
|
||||
required bytes row_processor_result = 1;
|
||||
}
|
||||
|
||||
service RowProcessorService {
|
||||
rpc Process(ProcessRequest) returns (ProcessResponse);
|
||||
}
|
|
@ -0,0 +1,84 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
syntax = "proto2";
|
||||
package hbase.pb;
|
||||
|
||||
option java_package = "org.apache.hadoop.hbase.shaded.protobuf.generated";
|
||||
option java_outer_classname = "VisibilityLabelsProtos";
|
||||
option java_generic_services = true;
|
||||
option java_generate_equals_and_hash = true;
|
||||
option optimize_for = SPEED;
|
||||
|
||||
import "Client.proto";
|
||||
|
||||
message VisibilityLabelsRequest {
|
||||
repeated VisibilityLabel visLabel = 1;
|
||||
}
|
||||
|
||||
message VisibilityLabel {
|
||||
required bytes label = 1;
|
||||
optional uint32 ordinal = 2;
|
||||
}
|
||||
|
||||
message VisibilityLabelsResponse {
|
||||
repeated RegionActionResult result = 1;
|
||||
}
|
||||
|
||||
message SetAuthsRequest {
|
||||
required bytes user = 1;
|
||||
repeated bytes auth = 2;
|
||||
}
|
||||
|
||||
message UserAuthorizations {
|
||||
required bytes user = 1;
|
||||
repeated uint32 auth = 2;
|
||||
}
|
||||
|
||||
message MultiUserAuthorizations {
|
||||
repeated UserAuthorizations userAuths = 1;
|
||||
}
|
||||
|
||||
message GetAuthsRequest {
|
||||
required bytes user = 1;
|
||||
}
|
||||
|
||||
message GetAuthsResponse {
|
||||
required bytes user = 1;
|
||||
repeated bytes auth = 2;
|
||||
}
|
||||
|
||||
message ListLabelsRequest {
|
||||
optional string regex = 1;
|
||||
}
|
||||
|
||||
message ListLabelsResponse {
|
||||
repeated bytes label = 1;
|
||||
}
|
||||
|
||||
service VisibilityLabelsService {
|
||||
rpc addLabels(VisibilityLabelsRequest)
|
||||
returns (VisibilityLabelsResponse);
|
||||
rpc setAuths(SetAuthsRequest)
|
||||
returns (VisibilityLabelsResponse);
|
||||
rpc clearAuths(SetAuthsRequest)
|
||||
returns (VisibilityLabelsResponse);
|
||||
rpc getAuths(GetAuthsRequest)
|
||||
returns (GetAuthsResponse);
|
||||
rpc listLabels(ListLabelsRequest)
|
||||
returns (ListLabelsResponse);
|
||||
}
|
|
@ -18,10 +18,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.rest.client;
|
||||
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.Service;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.io.UnsupportedEncodingException;
|
||||
|
@ -77,6 +73,10 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
|
||||
/**
|
||||
* HTable interface to remote tables accessed via REST gateway
|
||||
|
|
|
@ -18,15 +18,17 @@
|
|||
|
||||
package org.apache.hadoop.hbase.client.coprocessor;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.RowProcessor;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessRequest;
|
||||
import org.apache.hadoop.hbase.regionserver.RowProcessor;
|
||||
|
||||
import com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RowProcessorProtos.ProcessRequest;
|
||||
|
||||
/**
|
||||
* Convenience class that is used to make RowProcessorEndpoint invocations.
|
||||
* For example usage, refer TestRowProcessorEndpoint
|
||||
|
|
|
@ -21,24 +21,25 @@ import java.io.IOException;
|
|||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.util.Collections;
|
||||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessResponse;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.RowProcessorService;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.RowProcessor;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RowProcessorProtos.ProcessRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RowProcessorProtos.ProcessResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RowProcessorProtos.RowProcessorService;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
|
||||
/**
|
||||
* This class demonstrates how to implement atomic read-modify-writes
|
||||
|
|
|
@ -18,10 +18,11 @@
|
|||
|
||||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import com.google.protobuf.Service;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
/**
|
||||
* Coprocessor endpoints providing protobuf services should implement this
|
||||
|
|
|
@ -1,86 +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.coprocessor;
|
||||
|
||||
import com.google.protobuf.Service;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import java.util.Collections;
|
||||
|
||||
/**
|
||||
* Classes to help maintain backward compatibility with now deprecated {@link CoprocessorService}
|
||||
* and {@link SingletonCoprocessorService}.
|
||||
* From 2.0 onwards, implementors of coprocessor service should also implement the relevant
|
||||
* coprocessor class (For eg {@link MasterCoprocessor} for coprocessor service in master), and
|
||||
* override get*Service() method to return the {@link com.google.protobuf.Service} object.
|
||||
* To maintain backward compatibility with 1.0 implementation, we'll wrap implementation of
|
||||
* CoprocessorService/SingletonCoprocessorService in the new
|
||||
* {Master, Region, RegionServer}Coprocessor class.
|
||||
* Since there is no backward compatibility guarantee for Observers, we leave get*Observer() to
|
||||
* default which returns null.
|
||||
* This approach to maintain backward compatibility seems cleaner and more explicit.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@Deprecated
|
||||
public class CoprocessorServiceBackwardCompatiblity {
|
||||
|
||||
static public class MasterCoprocessorService implements MasterCoprocessor {
|
||||
|
||||
CoprocessorService service;
|
||||
|
||||
public MasterCoprocessorService(CoprocessorService service) {
|
||||
this.service = service;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Service> getServices() {
|
||||
return Collections.singleton(service.getService());
|
||||
}
|
||||
}
|
||||
|
||||
static public class RegionCoprocessorService implements RegionCoprocessor {
|
||||
|
||||
CoprocessorService service;
|
||||
|
||||
public RegionCoprocessorService(CoprocessorService service) {
|
||||
this.service = service;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Service> getServices() {
|
||||
return Collections.singleton(service.getService());
|
||||
}
|
||||
}
|
||||
|
||||
static public class RegionServerCoprocessorService implements RegionServerCoprocessor {
|
||||
|
||||
SingletonCoprocessorService service;
|
||||
|
||||
public RegionServerCoprocessorService(SingletonCoprocessorService service) {
|
||||
this.service = service;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<Service> getServices() {
|
||||
return Collections.singleton(service.getService());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -18,15 +18,14 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.Service;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
/**
|
||||
* Coprocessors implement this interface to observe and mediate endpoint invocations
|
||||
* on a region.
|
||||
|
|
|
@ -24,27 +24,28 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
import java.util.SortedSet;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.WrongRegionException;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
import com.google.protobuf.RpcCallback;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse;
|
||||
|
||||
|
||||
/**
|
||||
* This class demonstrates how to implement atomic multi row transactions using
|
||||
|
|
|
@ -18,10 +18,11 @@
|
|||
|
||||
package org.apache.hadoop.hbase.coprocessor;
|
||||
|
||||
import com.google.protobuf.Service;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
/**
|
||||
* Coprocessor endpoints registered once per server and providing protobuf services should implement
|
||||
|
|
|
@ -21,8 +21,7 @@ import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_SPLIT_COORDINATED
|
|||
import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;
|
||||
import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_COORDINATED_BY_ZK;
|
||||
import static org.apache.hadoop.hbase.util.DNS.MASTER_HOSTNAME_KEY;
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Service;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.lang.reflect.Constructor;
|
||||
|
@ -223,10 +222,14 @@ import org.eclipse.jetty.servlet.ServletHolder;
|
|||
import org.eclipse.jetty.webapp.WebAppContext;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import com.google.protobuf.Service;
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.util.List;
|
||||
|
@ -38,8 +37,6 @@ import org.apache.hadoop.hbase.client.SnapshotDescription;
|
|||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||
import org.apache.hadoop.hbase.coprocessor.BaseEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorServiceBackwardCompatiblity;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor;
|
||||
import org.apache.hadoop.hbase.coprocessor.HasMasterServices;
|
||||
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
|
||||
|
@ -64,6 +61,8 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
/**
|
||||
* Provides the coprocessor framework and environment for master oriented
|
||||
* operations. {@link HMaster} interacts with the loaded coprocessors
|
||||
|
@ -177,12 +176,6 @@ public class MasterCoprocessorHost
|
|||
try {
|
||||
if (MasterCoprocessor.class.isAssignableFrom(implClass)) {
|
||||
return implClass.asSubclass(MasterCoprocessor.class).getDeclaredConstructor().newInstance();
|
||||
} else if (CoprocessorService.class.isAssignableFrom(implClass)) {
|
||||
// For backward compatibility with old CoprocessorService impl which don't extend
|
||||
// MasterCoprocessor.
|
||||
CoprocessorService cs;
|
||||
cs = implClass.asSubclass(CoprocessorService.class).getDeclaredConstructor().newInstance();
|
||||
return new CoprocessorServiceBackwardCompatiblity.MasterCoprocessorService(cs);
|
||||
} else {
|
||||
LOG.error("{} is not of type MasterCoprocessor. Check the configuration of {}",
|
||||
implClass.getName(), CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY);
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import static org.apache.hadoop.hbase.master.MasterWalManager.META_FILTER;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.net.BindException;
|
||||
|
@ -32,7 +33,6 @@ import java.util.Map.Entry;
|
|||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
|
||||
|
@ -84,8 +84,6 @@ import org.apache.hadoop.hbase.procedure2.Procedure;
|
|||
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
|
||||
import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
|
||||
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
|
||||
import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
|
||||
import org.apache.hadoop.hbase.quotas.QuotaUtil;
|
||||
|
@ -108,7 +106,6 @@ import org.apache.hadoop.hbase.security.access.PermissionStorage;
|
|||
import org.apache.hadoop.hbase.security.access.ShadedAccessControlUtil;
|
||||
import org.apache.hadoop.hbase.security.access.UserPermission;
|
||||
import org.apache.hadoop.hbase.security.visibility.VisibilityController;
|
||||
|
||||
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -117,20 +114,24 @@ import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
|
|||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
|
||||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.ServiceDescriptor;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ResponseConverter;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.AccessControlService;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GetUserPermissionsResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AccessControlProtos.GrantRequest;
|
||||
|
@ -377,8 +378,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.Trans
|
|||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigRequest;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ReplicationProtos.UpdateReplicationPeerConfigResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
|
||||
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService;
|
||||
|
||||
/**
|
||||
* Implements the master RPC services.
|
||||
|
@ -922,14 +922,14 @@ public class MasterRpcServices extends RSRpcServices implements
|
|||
". Has it been enabled?");
|
||||
}
|
||||
|
||||
com.google.protobuf.Service service = master.coprocessorServiceHandlers.get(serviceName);
|
||||
com.google.protobuf.Descriptors.ServiceDescriptor serviceDesc = service.getDescriptorForType();
|
||||
com.google.protobuf.Descriptors.MethodDescriptor methodDesc =
|
||||
Service service = master.coprocessorServiceHandlers.get(serviceName);
|
||||
ServiceDescriptor serviceDesc = service.getDescriptorForType();
|
||||
MethodDescriptor methodDesc =
|
||||
CoprocessorRpcUtils.getMethodDescriptor(methodName, serviceDesc);
|
||||
|
||||
com.google.protobuf.Message execRequest =
|
||||
Message execRequest =
|
||||
CoprocessorRpcUtils.getRequest(service, methodDesc, call.getRequest());
|
||||
final com.google.protobuf.Message.Builder responseBuilder =
|
||||
final Message.Builder responseBuilder =
|
||||
service.getResponsePrototype(methodDesc).newBuilderForType();
|
||||
service.callMethod(methodDesc, execController, execRequest,
|
||||
(message) -> {
|
||||
|
@ -937,7 +937,7 @@ public class MasterRpcServices extends RSRpcServices implements
|
|||
responseBuilder.mergeFrom(message);
|
||||
}
|
||||
});
|
||||
com.google.protobuf.Message execResult = responseBuilder.build();
|
||||
Message execResult = responseBuilder.build();
|
||||
if (execController.getFailedOn() != null) {
|
||||
throw execController.getFailedOn();
|
||||
}
|
||||
|
@ -1985,21 +1985,20 @@ public class MasterRpcServices extends RSRpcServices implements
|
|||
|
||||
/**
|
||||
* Determines if there is a MasterCoprocessor deployed which implements
|
||||
* {@link org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService.Interface}.
|
||||
* {@link AccessControlService.Interface}.
|
||||
*/
|
||||
boolean hasAccessControlServiceCoprocessor(MasterCoprocessorHost cpHost) {
|
||||
return checkCoprocessorWithService(
|
||||
cpHost.findCoprocessors(MasterCoprocessor.class), AccessControlService.Interface.class);
|
||||
return checkCoprocessorWithService(cpHost.findCoprocessors(MasterCoprocessor.class),
|
||||
AccessControlService.Interface.class);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines if there is a MasterCoprocessor deployed which implements
|
||||
* {@link org.apache.hadoop.hbase.protobuf.generated.VisibilityLabelsProtos.VisibilityLabelsService.Interface}.
|
||||
* {@link VisibilityLabelsService.Interface}.
|
||||
*/
|
||||
boolean hasVisibilityLabelsServiceCoprocessor(MasterCoprocessorHost cpHost) {
|
||||
return checkCoprocessorWithService(
|
||||
cpHost.findCoprocessors(MasterCoprocessor.class),
|
||||
VisibilityLabelsService.Interface.class);
|
||||
return checkCoprocessorWithService(cpHost.findCoprocessors(MasterCoprocessor.class),
|
||||
VisibilityLabelsService.Interface.class);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import com.google.protobuf.Service;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.hbase.Server;
|
||||
|
@ -56,6 +55,7 @@ import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
|
|||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
/**
|
||||
* A curated subset of services provided by {@link HMaster}.
|
||||
|
|
|
@ -18,19 +18,19 @@
|
|||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.TableState;
|
||||
import org.apache.hadoop.hbase.client.TableState.State;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos;
|
||||
|
||||
/**
|
||||
* A subclass of TableStateManager that mirrors change in state out to zookeeper for hbase-1.x
|
||||
* clients to pick up; hbase-1.x clients read table state of zookeeper rather than from hbase:meta
|
||||
|
|
|
@ -22,14 +22,13 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
import com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
|
||||
/**
|
||||
* Base class for RowProcessor with some default implementations.
|
||||
|
|
|
@ -192,6 +192,11 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
|
|||
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
|
||||
import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.ServiceDescriptor;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
|
||||
|
@ -299,7 +304,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
new ConcurrentSkipListMap<>(Bytes.BYTES_RAWCOMPARATOR);
|
||||
|
||||
// TODO: account for each registered handler in HeapSize computation
|
||||
private Map<String, com.google.protobuf.Service> coprocessorServiceHandlers = Maps.newHashMap();
|
||||
private Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
|
||||
|
||||
// Track data size in all memstores
|
||||
private final MemStoreSizing memStoreSizing = new ThreadSafeMemStoreSizing();
|
||||
|
@ -8394,23 +8399,20 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
* @return {@code true} if the registration was successful, {@code false}
|
||||
* otherwise
|
||||
*/
|
||||
public boolean registerService(com.google.protobuf.Service instance) {
|
||||
/*
|
||||
* No stacking of instances is allowed for a single service name
|
||||
*/
|
||||
com.google.protobuf.Descriptors.ServiceDescriptor serviceDesc = instance.getDescriptorForType();
|
||||
public boolean registerService(Service instance) {
|
||||
// No stacking of instances is allowed for a single service name
|
||||
ServiceDescriptor serviceDesc = instance.getDescriptorForType();
|
||||
String serviceName = CoprocessorRpcUtils.getServiceName(serviceDesc);
|
||||
if (coprocessorServiceHandlers.containsKey(serviceName)) {
|
||||
LOG.error("Coprocessor service " + serviceName +
|
||||
" already registered, rejecting request from " + instance);
|
||||
" already registered, rejecting request from " + instance);
|
||||
return false;
|
||||
}
|
||||
|
||||
coprocessorServiceHandlers.put(serviceName, instance);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Registered coprocessor service: region=" +
|
||||
Bytes.toStringBinary(getRegionInfo().getRegionName()) +
|
||||
" service=" + serviceName);
|
||||
Bytes.toStringBinary(getRegionInfo().getRegionName()) + " service=" + serviceName);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
@ -8418,7 +8420,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
/**
|
||||
* Executes a single protocol buffer coprocessor endpoint {@link Service} method using
|
||||
* the registered protocol handlers. {@link Service} implementations must be registered via the
|
||||
* {@link #registerService(com.google.protobuf.Service)}
|
||||
* {@link #registerService(Service)}
|
||||
* method before they are available.
|
||||
*
|
||||
* @param controller an {@code RpcContoller} implementation to pass to the invoked service
|
||||
|
@ -8427,41 +8429,40 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
* @return a protocol buffer {@code Message} instance containing the method's result
|
||||
* @throws IOException if no registered service handler is found or an error
|
||||
* occurs during the invocation
|
||||
* @see #registerService(com.google.protobuf.Service)
|
||||
* @see #registerService(Service)
|
||||
*/
|
||||
public com.google.protobuf.Message execService(com.google.protobuf.RpcController controller,
|
||||
CoprocessorServiceCall call) throws IOException {
|
||||
public Message execService(RpcController controller, CoprocessorServiceCall call)
|
||||
throws IOException {
|
||||
String serviceName = call.getServiceName();
|
||||
com.google.protobuf.Service service = coprocessorServiceHandlers.get(serviceName);
|
||||
Service service = coprocessorServiceHandlers.get(serviceName);
|
||||
if (service == null) {
|
||||
throw new UnknownProtocolException(null, "No registered coprocessor service found for " +
|
||||
serviceName + " in region " + Bytes.toStringBinary(getRegionInfo().getRegionName()));
|
||||
}
|
||||
com.google.protobuf.Descriptors.ServiceDescriptor serviceDesc = service.getDescriptorForType();
|
||||
ServiceDescriptor serviceDesc = service.getDescriptorForType();
|
||||
|
||||
cpRequestsCount.increment();
|
||||
String methodName = call.getMethodName();
|
||||
com.google.protobuf.Descriptors.MethodDescriptor methodDesc =
|
||||
MethodDescriptor methodDesc =
|
||||
CoprocessorRpcUtils.getMethodDescriptor(methodName, serviceDesc);
|
||||
|
||||
com.google.protobuf.Message.Builder builder =
|
||||
Message.Builder builder =
|
||||
service.getRequestPrototype(methodDesc).newBuilderForType();
|
||||
|
||||
org.apache.hadoop.hbase.protobuf.ProtobufUtil.mergeFrom(builder,
|
||||
ProtobufUtil.mergeFrom(builder,
|
||||
call.getRequest().toByteArray());
|
||||
com.google.protobuf.Message request =
|
||||
Message request =
|
||||
CoprocessorRpcUtils.getRequest(service, methodDesc, call.getRequest());
|
||||
|
||||
if (coprocessorHost != null) {
|
||||
request = coprocessorHost.preEndpointInvocation(service, methodName, request);
|
||||
}
|
||||
|
||||
final com.google.protobuf.Message.Builder responseBuilder =
|
||||
final Message.Builder responseBuilder =
|
||||
service.getResponsePrototype(methodDesc).newBuilderForType();
|
||||
service.callMethod(methodDesc, controller, request,
|
||||
new com.google.protobuf.RpcCallback<com.google.protobuf.Message>() {
|
||||
service.callMethod(methodDesc, controller, request, new RpcCallback<Message>() {
|
||||
@Override
|
||||
public void run(com.google.protobuf.Message message) {
|
||||
public void run(Message message) {
|
||||
if (message != null) {
|
||||
responseBuilder.mergeFrom(message);
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_SPLIT_WAL_MAX_SPL
|
|||
import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_COORDINATED_BY_ZK;
|
||||
import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_MAX_SPLITTER;
|
||||
import static org.apache.hadoop.hbase.util.DNS.RS_HOSTNAME_KEY;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.Thread.UncaughtExceptionHandler;
|
||||
import java.lang.management.MemoryType;
|
||||
|
@ -188,6 +189,7 @@ import org.apache.zookeeper.KeeperException;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import sun.misc.Signal;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
|
||||
import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
|
||||
|
@ -195,10 +197,15 @@ import org.apache.hbase.thirdparty.com.google.common.cache.Cache;
|
|||
import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
|
||||
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.BlockingRpcChannel;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.ServiceDescriptor;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.ServiceException;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
|
||||
|
@ -433,7 +440,7 @@ public class HRegionServer extends HasThread implements
|
|||
/** The nonce manager chore. */
|
||||
private ScheduledChore nonceManagerChore;
|
||||
|
||||
private Map<String, com.google.protobuf.Service> coprocessorServiceHandlers = Maps.newHashMap();
|
||||
private Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
|
||||
|
||||
/**
|
||||
* The server name the Master sees us as. Its made from the hostname the
|
||||
|
@ -762,22 +769,20 @@ public class HRegionServer extends HasThread implements
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean registerService(com.google.protobuf.Service instance) {
|
||||
/*
|
||||
* No stacking of instances is allowed for a single executorService name
|
||||
*/
|
||||
com.google.protobuf.Descriptors.ServiceDescriptor serviceDesc =
|
||||
instance.getDescriptorForType();
|
||||
public boolean registerService(Service instance) {
|
||||
// No stacking of instances is allowed for a single executorService name
|
||||
ServiceDescriptor serviceDesc = instance.getDescriptorForType();
|
||||
String serviceName = CoprocessorRpcUtils.getServiceName(serviceDesc);
|
||||
if (coprocessorServiceHandlers.containsKey(serviceName)) {
|
||||
LOG.error("Coprocessor executorService " + serviceName
|
||||
+ " already registered, rejecting request from " + instance);
|
||||
LOG.error("Coprocessor executorService " + serviceName +
|
||||
" already registered, rejecting request from " + instance);
|
||||
return false;
|
||||
}
|
||||
|
||||
coprocessorServiceHandlers.put(serviceName, instance);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Registered regionserver coprocessor executorService: executorService=" + serviceName);
|
||||
LOG.debug(
|
||||
"Registered regionserver coprocessor executorService: executorService=" + serviceName);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
@ -3563,25 +3568,25 @@ public class HRegionServer extends HasThread implements
|
|||
ServerRpcController serviceController = new ServerRpcController();
|
||||
CoprocessorServiceCall call = serviceRequest.getCall();
|
||||
String serviceName = call.getServiceName();
|
||||
com.google.protobuf.Service service = coprocessorServiceHandlers.get(serviceName);
|
||||
Service service = coprocessorServiceHandlers.get(serviceName);
|
||||
if (service == null) {
|
||||
throw new UnknownProtocolException(null, "No registered coprocessor executorService found for " +
|
||||
serviceName);
|
||||
}
|
||||
com.google.protobuf.Descriptors.ServiceDescriptor serviceDesc =
|
||||
ServiceDescriptor serviceDesc =
|
||||
service.getDescriptorForType();
|
||||
|
||||
String methodName = call.getMethodName();
|
||||
com.google.protobuf.Descriptors.MethodDescriptor methodDesc =
|
||||
MethodDescriptor methodDesc =
|
||||
serviceDesc.findMethodByName(methodName);
|
||||
if (methodDesc == null) {
|
||||
throw new UnknownProtocolException(service.getClass(), "Unknown method " + methodName +
|
||||
" called on executorService " + serviceName);
|
||||
}
|
||||
|
||||
com.google.protobuf.Message request =
|
||||
Message request =
|
||||
CoprocessorRpcUtils.getRequest(service, methodDesc, call.getRequest());
|
||||
final com.google.protobuf.Message.Builder responseBuilder =
|
||||
final Message.Builder responseBuilder =
|
||||
service.getResponsePrototype(methodDesc).newBuilderForType();
|
||||
service.callMethod(methodDesc, serviceController, request, message -> {
|
||||
if (message != null) {
|
||||
|
|
|
@ -871,7 +871,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
}
|
||||
} else if (action.hasServiceCall()) {
|
||||
hasResultOrException = true;
|
||||
com.google.protobuf.Message result =
|
||||
Message result =
|
||||
execServiceOnRegion(region, action.getServiceCall());
|
||||
ClientProtos.CoprocessorServiceResult.Builder serviceResultBuilder =
|
||||
ClientProtos.CoprocessorServiceResult.newBuilder();
|
||||
|
@ -2493,7 +2493,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
checkOpen();
|
||||
requestCount.increment();
|
||||
HRegion region = getRegion(request.getRegion());
|
||||
com.google.protobuf.Message result = execServiceOnRegion(region, request.getCall());
|
||||
Message result = execServiceOnRegion(region, request.getCall());
|
||||
CoprocessorServiceResponse.Builder builder = CoprocessorServiceResponse.newBuilder();
|
||||
builder.setRegion(RequestConverter.buildRegionSpecifier(
|
||||
RegionSpecifierType.REGION_NAME, region.getRegionInfo().getRegionName()));
|
||||
|
@ -2516,7 +2516,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
|
|||
return new Path(filePaths.get(0)).getFileSystem(regionServer.getConfiguration());
|
||||
}
|
||||
|
||||
private com.google.protobuf.Message execServiceOnRegion(HRegion region,
|
||||
private Message execServiceOnRegion(HRegion region,
|
||||
final ClientProtos.CoprocessorServiceCall serviceCall) throws IOException {
|
||||
// ignore the passed in controller (from the serialized call)
|
||||
ServerRpcController execController = new ServerRpcController();
|
||||
|
|
|
@ -18,9 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.Service;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -57,8 +54,6 @@ import org.apache.hadoop.hbase.coprocessor.BaseEnvironment;
|
|||
import org.apache.hadoop.hbase.coprocessor.BulkLoadObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorServiceBackwardCompatiblity;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor;
|
||||
import org.apache.hadoop.hbase.coprocessor.EndpointObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.HasRegionServerServices;
|
||||
|
@ -86,6 +81,8 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
import org.apache.hbase.thirdparty.org.apache.commons.collections4.map.AbstractReferenceMap;
|
||||
import org.apache.hbase.thirdparty.org.apache.commons.collections4.map.ReferenceMap;
|
||||
|
||||
|
@ -435,12 +432,6 @@ public class RegionCoprocessorHost
|
|||
try {
|
||||
if (RegionCoprocessor.class.isAssignableFrom(implClass)) {
|
||||
return implClass.asSubclass(RegionCoprocessor.class).getDeclaredConstructor().newInstance();
|
||||
} else if (CoprocessorService.class.isAssignableFrom(implClass)) {
|
||||
// For backward compatibility with old CoprocessorService impl which don't extend
|
||||
// RegionCoprocessor.
|
||||
CoprocessorService cs;
|
||||
cs = implClass.asSubclass(CoprocessorService.class).getDeclaredConstructor().newInstance();
|
||||
return new CoprocessorServiceBackwardCompatiblity.RegionCoprocessorService(cs);
|
||||
} else {
|
||||
LOG.error("{} is not of type RegionCoprocessor. Check the configuration of {}",
|
||||
implClass.getName(), CoprocessorHost.REGION_COPROCESSOR_CONF_KEY);
|
||||
|
|
|
@ -20,23 +20,18 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
|
||||
import com.google.protobuf.Service;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.SharedConnection;
|
||||
import org.apache.hadoop.hbase.coprocessor.BaseEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorServiceBackwardCompatiblity;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor;
|
||||
import org.apache.hadoop.hbase.coprocessor.HasRegionServerServices;
|
||||
import org.apache.hadoop.hbase.coprocessor.MetricsCoprocessor;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessor;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.coprocessor.RegionServerObserver;
|
||||
import org.apache.hadoop.hbase.coprocessor.SingletonCoprocessorService;
|
||||
import org.apache.hadoop.hbase.metrics.MetricRegistry;
|
||||
import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
|
@ -44,6 +39,8 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class RegionServerCoprocessorHost extends
|
||||
CoprocessorHost<RegionServerCoprocessor, RegionServerCoprocessorEnvironment> {
|
||||
|
@ -87,12 +84,6 @@ public class RegionServerCoprocessorHost extends
|
|||
if (RegionServerCoprocessor.class.isAssignableFrom(implClass)) {
|
||||
return implClass.asSubclass(RegionServerCoprocessor.class).getDeclaredConstructor()
|
||||
.newInstance();
|
||||
} else if (SingletonCoprocessorService.class.isAssignableFrom(implClass)) {
|
||||
// For backward compatibility with old CoprocessorService impl which don't extend
|
||||
// RegionCoprocessor.
|
||||
SingletonCoprocessorService tmp = implClass.asSubclass(SingletonCoprocessorService.class)
|
||||
.getDeclaredConstructor().newInstance();
|
||||
return new CoprocessorServiceBackwardCompatiblity.RegionServerCoprocessorService(tmp);
|
||||
} else {
|
||||
LOG.error("{} is not of type RegionServerCoprocessor. Check the configuration of {}",
|
||||
implClass.getName(), CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY);
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import com.google.protobuf.Service;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
@ -45,6 +44,8 @@ import org.apache.hadoop.hbase.security.access.ZKPermissionWatcher;
|
|||
import org.apache.hadoop.hbase.wal.WAL;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Service;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
|
||||
|
||||
/**
|
||||
|
|
|
@ -21,15 +21,15 @@ import java.io.IOException;
|
|||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.wal.WALEdit;
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
import org.apache.yetus.audience.InterfaceStability;
|
||||
|
||||
import org.apache.hbase.thirdparty.com.google.protobuf.Message;
|
||||
|
||||
import com.google.protobuf.Message;
|
||||
|
||||
/**
|
||||
* Defines the procedures to atomically perform multiple scans and mutations
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue