From fbfbad4e4d51b0194f62c13b807cf75ec0013ff2 Mon Sep 17 00:00:00 2001 From: Gary Helmling Date: Tue, 18 Sep 2012 06:32:57 +0000 Subject: [PATCH] HBASE-5448 Support dynamic coprocessor endpoints with protobuf based RPC git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1387001 13f79535-47bb-0310-9956-ffa450edef68 --- .../hbase/client/HConnectionManager.java | 1 + .../apache/hadoop/hbase/client/HTable.java | 86 + .../hadoop/hbase/client/HTableInterface.java | 93 + .../hadoop/hbase/client/HTablePool.java | 22 + .../hbase/client/coprocessor/Batch.java | 2 + .../hadoop/hbase/client/coprocessor/Exec.java | 3 + .../hbase/client/coprocessor/ExecResult.java | 3 + .../client/coprocessor/package-info.java | 205 +- .../hbase/coprocessor/CoprocessorHost.java | 29 +- .../hbase/coprocessor/CoprocessorService.java | 33 + .../coprocessor/example/RowCountEndpoint.java | 164 + .../example/generated/ExampleProtos.java | 1034 +++++ .../hbase/coprocessor/package-info.java | 120 +- .../hadoop/hbase/ipc/BlockingRpcCallback.java | 73 + .../hadoop/hbase/ipc/CoprocessorProtocol.java | 3 + .../hbase/ipc/CoprocessorRpcChannel.java | 132 + .../hadoop/hbase/ipc/ExecRPCInvoker.java | 3 + .../hadoop/hbase/ipc/ServerRpcController.java | 124 + .../hadoop/hbase/protobuf/ProtobufUtil.java | 50 +- .../hbase/protobuf/ResponseConverter.java | 33 + .../generated/AccessControlProtos.java | 3941 ++++++++++++++++- .../protobuf/generated/ClientProtos.java | 2128 ++++++++- .../hadoop/hbase/regionserver/HRegion.java | 95 +- .../hbase/regionserver/HRegionServer.java | 30 + .../regionserver/RegionCoprocessorHost.java | 7 +- .../hbase/rest/client/RemoteHTable.java | 22 + .../security/access/AccessController.java | 97 +- .../src/main/protobuf/AccessControl.proto | 45 + hbase-server/src/main/protobuf/Client.proto | 20 + hbase-server/src/main/protobuf/Examples.proto | 37 + .../ColumnAggregationEndpoint.java | 13 +- .../ProtobufCoprocessorService.java | 77 + .../coprocessor/TestCoprocessorEndpoint.java | 141 +- .../example/TestRowCountEndpoint.java | 111 + .../hadoop/hbase/master/MockRegionServer.java | 7 + .../security/access/TestAccessController.java | 132 +- 36 files changed, 8819 insertions(+), 297 deletions(-) create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorService.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/generated/ExampleProtos.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcCallback.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java create mode 100644 hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcController.java create mode 100644 hbase-server/src/main/protobuf/Examples.proto create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ProtobufCoprocessorService.java create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java index 3657afc59f3..1665e5db6bf 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java @@ -2140,6 +2140,7 @@ public class HConnectionManager { * @param the callable's return type * @throws IOException */ + @Deprecated public void processExecs( final Class protocol, List rows, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTable.java index 99836cd6b6d..1b32d42e42b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -32,11 +32,17 @@ import java.util.List; import java.util.Map; import java.util.NavigableMap; import java.util.TreeMap; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import com.google.protobuf.Service; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; @@ -56,6 +62,7 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.io.DataInputInputStream; import org.apache.hadoop.hbase.ipc.CoprocessorProtocol; +import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.ExecRPCInvoker; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.RequestConverter; @@ -1318,6 +1325,7 @@ public class HTable implements HTableInterface { * {@inheritDoc} */ @Override + @Deprecated public T coprocessorProxy( Class protocol, byte[] row) { return (T)Proxy.newProxyInstance(this.getClass().getClassLoader(), @@ -1329,10 +1337,18 @@ public class HTable implements HTableInterface { row)); } + /** + * {@inheritDoc} + */ + public CoprocessorRpcChannel coprocessorService(byte[] row) { + return new CoprocessorRpcChannel(connection, tableName, row); + } + /** * {@inheritDoc} */ @Override + @Deprecated public Map coprocessorExec( Class protocol, byte[] startKey, byte[] endKey, Batch.Call callable) @@ -1353,6 +1369,7 @@ public class HTable implements HTableInterface { * {@inheritDoc} */ @Override + @Deprecated public void coprocessorExec( Class protocol, byte[] startKey, byte[] endKey, Batch.Call callable, Batch.Callback callback) @@ -1364,6 +1381,75 @@ public class HTable implements HTableInterface { callback); } + /** + * {@inheritDoc} + */ + @Override + public Map coprocessorService(final Class service, + byte[] startKey, byte[] endKey, final Batch.Call callable) + throws ServiceException, Throwable { + final Map results = new ConcurrentSkipListMap(Bytes.BYTES_COMPARATOR); + coprocessorService(service, startKey, endKey, callable, new Batch.Callback() { + public void update(byte[] region, byte[] row, R value) { + if (value == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Call to " + service.getName() + + " received NULL value from Batch.Call for region " + Bytes.toStringBinary(region)); + } + } else { + results.put(region, value); + } + } + }); + return results; + } + + /** + * {@inheritDoc} + */ + @Override + public void coprocessorService(final Class service, + byte[] startKey, byte[] endKey, final Batch.Call callable, + final Batch.Callback callback) throws ServiceException, Throwable { + + // get regions covered by the row range + List keys = getStartKeysInRange(startKey, endKey); + + Map> futures = + new TreeMap>(Bytes.BYTES_COMPARATOR); + for (final byte[] r : keys) { + final CoprocessorRpcChannel channel = + new CoprocessorRpcChannel(connection, tableName, r); + Future future = pool.submit( + new Callable() { + public R call() throws Exception { + T instance = ProtobufUtil.newServiceStub(service, channel); + R result = callable.call(instance); + byte[] region = channel.getLastRegion(); + if (callback != null) { + callback.update(region, r, result); + } + return result; + } + }); + futures.put(r, future); + } + for (Map.Entry> e : futures.entrySet()) { + try { + e.getValue().get(); + } catch (ExecutionException ee) { + LOG.warn("Error calling coprocessor service " + service.getName() + " for row " + + Bytes.toStringBinary(e.getKey()), ee); + throw ee.getCause(); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new InterruptedIOException("Interrupted calling coprocessor service " + service.getName() + + " for row " + Bytes.toStringBinary(e.getKey())) + .initCause(ie); + } + } + } + private List getStartKeysInRange(byte[] start, byte[] end) throws IOException { Pair startEndKeys = getStartEndKeys(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java index c6681127f44..0927682cdd7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java @@ -23,6 +23,8 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import com.google.protobuf.Service; +import com.google.protobuf.ServiceException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; @@ -30,6 +32,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.ipc.CoprocessorProtocol; +import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; /** * Used to communicate with a single HBase table. @@ -425,7 +428,9 @@ public interface HTableInterface extends Closeable { * @param protocol The class or interface defining the remote protocol * @param row The row key used to identify the remote region location * @return A CoprocessorProtocol instance + * @deprecated since 0.96. Use {@link HTableInterface#coprocessorService(byte[])} instead. */ + @Deprecated T coprocessorProxy(Class protocol, byte[] row); /** @@ -450,7 +455,11 @@ public interface HTableInterface extends Closeable { * method * @return a Map of region names to * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call(Object)} return values + * + * @deprecated since 0.96. Use + * {@link HTableInterface#coprocessorService(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call)} instead. */ + @Deprecated Map coprocessorExec( Class protocol, byte[] startKey, byte[] endKey, Batch.Call callable) throws IOException, Throwable; @@ -486,12 +495,96 @@ public interface HTableInterface extends Closeable { * @param Return type for the * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call(Object)} * method + * + * @deprecated since 0.96. + * Use {@link HTableInterface#coprocessorService(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call, org.apache.hadoop.hbase.client.coprocessor.Batch.Callback)} instead. */ + @Deprecated void coprocessorExec( Class protocol, byte[] startKey, byte[] endKey, Batch.Call callable, Batch.Callback callback) throws IOException, Throwable; + /** + * Creates and returns a {@link com.google.protobuf.RpcChannel} instance connected to the + * table region containing the specified row. The row given does not actually have + * to exist. Whichever region would contain the row based on start and end keys will + * be used. Note that the {@code row} parameter is also not passed to the + * coprocessor handler registered for this protocol, unless the {@code row} + * is separately passed as an argument in the service request. The parameter + * here is only used to locate the region used to handle the call. + * + *

+ * The obtained {@link com.google.protobuf.RpcChannel} instance can be used to access a published + * coprocessor {@link com.google.protobuf.Service} using standard protobuf service invocations: + *

+ * + *
+ *
+   * CoprocessorRpcChannel channel = myTable.coprocessorService(rowkey);
+   * MyService.BlockingInterface service = MyService.newBlockingStub(channel);
+   * MyCallRequest request = MyCallRequest.newBuilder()
+   *     ...
+   *     .build();
+   * MyCallResponse response = service.myCall(null, request);
+   * 
+ * + * @param row The row key used to identify the remote region location + * @return A CoprocessorRpcChannel instance + */ + CoprocessorRpcChannel coprocessorService(byte[] row); + + /** + * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table + * region spanning the range from the {@code startKey} row to {@code endKey} row (inclusive), + * and invokes the passed {@link Batch.Call#call(Object)} method with each {@link Service} + * instance. + * + * @param service the protocol buffer {@code Service} implementation to call + * @param startKey start region selection with region containing this row. If {@code null}, the + * selection will start with the first table region. + * @param endKey select regions up to and including the region containing this row. + * If {@code null}, selection will continue through the last table region. + * @param callable this instance's {@link Batch.Call#call(Object)} method will be invoked once + * per table region, using the {@link Service} instance connected to that region. + * @param the {@link Service} subclass to connect to + * @param Return type for the {@code callable} parameter's + * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call(Object)} method + * @return a map of result values keyed by region name + */ + Map coprocessorService(final Class service, + byte[] startKey, byte[] endKey, final Batch.Call callable) + throws ServiceException, Throwable; + + /** + * Creates an instance of the given {@link com.google.protobuf.Service} subclass for each table + * region spanning the range from the {@code startKey} row to {@code endKey} row (inclusive), + * and invokes the passed {@link Batch.Call#call(Object)} method with each {@link Service} + * instance. + * + *

+ * The given + * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[], byte[], Object)} + * method will be called with the return value from each region's {@link Batch.Call#call(Object)} + * invocation. + *

+ * + * @param service the protocol buffer {@code Service} implementation to call + * @param startKey start region selection with region containing this row. If {@code null}, the + * selection will start with the first table region. + * @param endKey select regions up to and including the region containing this row. + * If {@code null}, selection will continue through the last table region. + * @param callable this instance's {@link Batch.Call#call(Object)} method will be invoked once + * per table region, using the {@link Service} instance connected to that region. + * @param callback + * @param the {@link Service} subclass to connect to + * @param Return type for the {@code callable} parameter's + * {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call(Object)} method + */ + void coprocessorService(final Class service, + byte[] startKey, byte[] endKey, final Batch.Call callable, + final Batch.Callback callback) throws ServiceException, Throwable; + /** * See {@link #setAutoFlush(boolean, boolean)} * diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java index 7e82078087e..3dc5b493cd2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java @@ -24,6 +24,8 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import com.google.protobuf.Service; +import com.google.protobuf.ServiceException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; @@ -32,6 +34,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback; import org.apache.hadoop.hbase.ipc.CoprocessorProtocol; +import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.PoolMap; import org.apache.hadoop.hbase.util.PoolMap.PoolType; @@ -492,6 +495,25 @@ public class HTablePool implements Closeable { table.coprocessorExec(protocol, startKey, endKey, callable, callback); } + @Override + public CoprocessorRpcChannel coprocessorService(byte[] row) { + return table.coprocessorService(row); + } + + @Override + public Map coprocessorService(Class service, + byte[] startKey, byte[] endKey, Batch.Call callable) + throws ServiceException, Throwable { + return table.coprocessorService(service, startKey, endKey, callable); + } + + @Override + public void coprocessorService(Class service, + byte[] startKey, byte[] endKey, Batch.Call callable, Callback callback) + throws ServiceException, Throwable { + table.coprocessorService(service, startKey, endKey, callable, callback); + } + @Override public String toString() { return "PooledHTable{" + ", table=" + table + '}'; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java index 5e1feb7da55..ed150cfe532 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Batch.java @@ -67,6 +67,7 @@ public abstract class Batch { * @see Batch#forMethod(java.lang.reflect.Method, Object...) * @see org.apache.hadoop.hbase.client.HTable#coprocessorExec(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call, org.apache.hadoop.hbase.client.coprocessor.Batch.Callback) */ + @Deprecated public static Call forMethod( final Class protocol, final String method, final Object... args) throws NoSuchMethodException { @@ -100,6 +101,7 @@ public abstract class Batch { * return the results * @see org.apache.hadoop.hbase.client.HTable#coprocessorExec(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call, org.apache.hadoop.hbase.client.coprocessor.Batch.Callback) */ + @Deprecated public static Call forMethod( final Method method, final Object... args) { return new Call() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Exec.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Exec.java index 063a5af8eac..72c3395fd9a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Exec.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/coprocessor/Exec.java @@ -51,7 +51,10 @@ import java.lang.reflect.Method; * @see ExecResult * @see org.apache.hadoop.hbase.client.HTable#coprocessorExec(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call) * @see org.apache.hadoop.hbase.client.HTable#coprocessorExec(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call, org.apache.hadoop.hbase.client.coprocessor.Batch.Callback) + * @deprecated since 0.96.0. See {@link org.apache.hadoop.hbase.client.HTable#coprocessorService(byte[])} + * or related methods instead. */ +@Deprecated @InterfaceAudience.Public @InterfaceStability.Evolving public class Exec extends Invocation implements Row { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/coprocessor/ExecResult.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/coprocessor/ExecResult.java index abe4e7c87bc..687eabef3c5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/coprocessor/ExecResult.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/coprocessor/ExecResult.java @@ -46,7 +46,10 @@ import java.io.Serializable; * @see Exec * @see org.apache.hadoop.hbase.client.HTable#coprocessorExec(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call) * @see org.apache.hadoop.hbase.client.HTable#coprocessorExec(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call, org.apache.hadoop.hbase.client.coprocessor.Batch.Callback) + * @deprecated since 0.96.0. See {@link org.apache.hadoop.hbase.client.HTable#coprocessorService(byte[])} + * or related methods instead. */ +@Deprecated @InterfaceAudience.Public @InterfaceStability.Evolving public class ExecResult implements Writable { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/coprocessor/package-info.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/coprocessor/package-info.java index 97aa94640de..edb3c221d1a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/client/coprocessor/package-info.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/client/coprocessor/package-info.java @@ -37,30 +37,42 @@ protocols.

In order to provide a custom RPC protocol to clients, a coprocessor implementation -defines an interface that extends {@link org.apache.hadoop.hbase.ipc.CoprocessorProtocol}. -The interface can define any methods that the coprocessor wishes to expose. -Using this protocol, you can communicate with the coprocessor instances via -the {@link org.apache.hadoop.hbase.client.HTable#coprocessorProxy(Class, byte[])} and -{@link org.apache.hadoop.hbase.client.HTable#coprocessorExec(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call, org.apache.hadoop.hbase.client.coprocessor.Batch.Callback)} +must: +

    +
  • Define a protocol buffer Service and supporting Message types for the RPC methods. + See the + protocol buffer guide + for more details on defining services.
  • +
  • Generate the Service and Message code using the protoc compiler
  • +
  • Implement the generated Service interface in your coprocessor class and implement the + {@link org.apache.hadoop.hbase.coprocessor.CoprocessorService} interface. The + {@link org.apache.hadoop.hbase.coprocessor.CoprocessorService#getService()} + method should return a reference to the Endpoint's protocol buffer Service instance. +
+Clients may then call the defined service methods on coprocessor instances via +the {@link org.apache.hadoop.hbase.client.HTable#coprocessorService(byte[])}, +{@link org.apache.hadoop.hbase.client.HTable#coprocessorService(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call)}, and +{@link org.apache.hadoop.hbase.client.HTable#coprocessorService(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call, org.apache.hadoop.hbase.client.coprocessor.Batch.Callback)} methods.

-Since {@link org.apache.hadoop.hbase.ipc.CoprocessorProtocol} instances are -associated with individual regions within the table, the client RPC calls -must ultimately identify which regions should be used in the CoprocessorProtocol +Since coprocessor Service instances are associated with individual regions within the table, +the client RPC calls must ultimately identify which regions should be used in the Service method invocations. Since regions are seldom handled directly in client code and the region names may change over time, the coprocessor RPC calls use row keys to identify which regions should be used for the method invocations. Clients -can call CoprocessorProtocol methods against either: +can call coprocessor Service methods against either:

  • a single region - calling - {@link org.apache.hadoop.hbase.client.HTable#coprocessorProxy(Class, byte[])} - with a single row key. This returns a dynamic proxy of the CoprocessorProtocol - interface which uses the region containing the given row key (even if the - row does not exist) as the RPC endpoint.
  • + {@link org.apache.hadoop.hbase.client.HTable#coprocessorService(byte[])} + with a single row key. This returns a {@link org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel} + instance which communicates with the region containing the given row key (even if the + row does not exist) as the RPC endpoint. Clients can then use the {@code CoprocessorRpcChannel} + instance in creating a new Service stub to call RPC methods on the region's coprocessor.
  • a range of regions - calling - {@link org.apache.hadoop.hbase.client.HTable#coprocessorExec(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call, org.apache.hadoop.hbase.client.coprocessor.Batch.Callback)} + {@link org.apache.hadoop.hbase.client.HTable#coprocessorService(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call)} + or {@link org.apache.hadoop.hbase.client.HTable#coprocessorService(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call, org.apache.hadoop.hbase.client.coprocessor.Batch.Callback)} with a starting row key and an ending row key. All regions in the table from the region containing the start row key to the region containing the end row key (inclusive), will we used as the RPC endpoints.
  • @@ -68,17 +80,16 @@ can call CoprocessorProtocol methods against either:

    Note that the row keys passed as parameters to the HTable -methods are not passed to the CoprocessorProtocol implementations. +methods are not passed directly to the coprocessor Service implementations. They are only used to identify the regions for endpoints of the remote calls.

    The {@link org.apache.hadoop.hbase.client.coprocessor.Batch} class defines two -interfaces used for CoprocessorProtocol invocations against -multiple regions. Clients implement {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call} to -call methods of the actual CoprocessorProtocol instance. The interface's -call() method will be called once per selected region, passing the -CoprocessorProtocol instance for the region as a parameter. Clients +interfaces used for coprocessor Service invocations against multiple regions. Clients implement +{@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call} to call methods of the actual +coprocessor Service instance. The interface's call() method will be called once +per selected region, passing the Service instance for the region as a parameter. Clients can optionally implement {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback} to be notified of the results from each region invocation as they complete. The instance's {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Callback#update(byte[], byte[], Object)} @@ -88,112 +99,128 @@ return value from each region.

    Example usage

    -To start with, let's use a fictitious coprocessor, RowCountCoprocessor +To start with, let's use a fictitious coprocessor, RowCountEndpoint that counts the number of rows and key-values in each region where it is running. -For clients to query this information, the coprocessor defines and implements -the following {@link org.apache.hadoop.hbase.ipc.CoprocessorProtocol} extension -interface: +For clients to query this information, the coprocessor defines the following protocol buffer +service:

    -public interface RowCountProtocol extends CoprocessorProtocol {
    -  long getRowCount();
    -  long getRowCount(Filter filt);
    -  long getKeyValueCount();
    +message CountRequest {
    +}
    +
    +message CountResponse {
    +  required int64 count = 1 [default = 0];
    +}
    +
    +service RowCountService {
    +  rpc getRowCount(CountRequest)
    +    returns (CountResponse);
    +  rpc getKeyValueCount(CountRequest)
    +    returns (CountResponse);
     }
     

    -Now we need a way to access the results that RowCountCoprocessor +Next run the protoc compiler on the .proto file to generate Java code for the Service interface. +The generated {@code RowCountService} interface should look something like: +

    +
    +
    +public static abstract class RowCountService
    +  implements com.google.protobuf.Service {
    +  ...
    +  public interface Interface {
    +    public abstract void getRowCount(
    +        com.google.protobuf.RpcController controller,
    +        org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request,
    +        com.google.protobuf.RpcCallback done);
    +
    +    public abstract void getKeyValueCount(
    +        com.google.protobuf.RpcController controller,
    +        org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request,
    +        com.google.protobuf.RpcCallback done);
    +  }
    +}
    +
    + +

    +Our coprocessor Service will need to implement this interface and the {@link org.apache.hadoop.hbase.coprocessor.CoprocessorService} +in order to be registered correctly as an endpoint. For the sake of simplicity the server-side +implementation is omitted. To see the implementing code, please see the +{@link org.apache.hadoop.hbase.coprocessor.example.RowCountEndpoint} class in the HBase source code. +

    + +

    +Now we need a way to access the results that RowCountService is making available. If we want to find the row count for all regions, we could use:

    -HTable table = new HTable("mytable");
    -// find row count keyed by region name
    -Map results = table.coprocessorExec(
    -    RowCountProtocol.class, // the protocol interface we're invoking
    -    null, null,             // start and end row keys
    -    new Batch.Call() {
    -       public Long call(RowCountProtocol counter) {
    -         return counter.getRowCount();
    -       }
    -     });
    +HTable table = new HTable(conf, "mytable");
    +final ExampleProtos.CountRequest request = ExampleProtos.CountRequest.getDefaultInstance();
    +Map results = table.coprocessorService(
    +    ExampleProtos.RowCountService.class, // the protocol interface we're invoking
    +    null, null,                          // start and end row keys
    +    new Batch.Call() {
    +        public Long call(ExampleProtos.RowCountService counter) throws IOException {
    +          BlockingRpcCallback rpcCallback =
    +              new BlockingRpcCallback();
    +          counter.getRowCount(null, request, rpcCallback);
    +          ExampleProtos.CountResponse response = rpcCallback.get();
    +          return response.hasCount() ? response.getCount() : 0;
    +        }
    +    });
     

    This will return a java.util.Map of the counter.getRowCount() -result for the RowCountCoprocessor instance running in each region +result for the RowCountService instance running in each region of mytable, keyed by the region name.

    By implementing {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call} -as an anonymous class, we can invoke RowCountProtocol methods +as an anonymous class, we can invoke RowCountService methods directly against the {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call#call(Object)} -method's argument. Calling {@link org.apache.hadoop.hbase.client.HTable#coprocessorExec(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call)} +method's argument. Calling {@link org.apache.hadoop.hbase.client.HTable#coprocessorService(Class, byte[], byte[], org.apache.hadoop.hbase.client.coprocessor.Batch.Call)} will take care of invoking Batch.Call.call() against our anonymous class -with the RowCountCoprocessor instance for each table region. +with the RowCountService instance for each table region.

    -For this simple case, where we only want to obtain the result from a single -CoprocessorProtocol method, there's also a bit of syntactic sugar -we can use to cut down on the amount of code required: -

    - -
    -
    -HTable table = new HTable("mytable");
    -Batch.Call call = Batch.forMethod(RowCountProtocol.class, "getRowCount");
    -Map results = table.coprocessorExec(RowCountProtocol.class, null, null, call);
    -
    - -

    -{@link org.apache.hadoop.hbase.client.coprocessor.Batch#forMethod(Class, String, Object...)} -is a simple factory method that will return a {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call} -instance that will call RowCountProtocol.getRowCount() for us -using reflection. -

    - -

    -However, if you want to perform additional processing on the results, -implementing {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call} -directly will provide more power and flexibility. For example, if you would +Implementing {@link org.apache.hadoop.hbase.client.coprocessor.Batch.Call} also allows you to +perform additional processing against each region's Service instance. For example, if you would like to combine row count and key-value count for each region:

    -HTable table = new HTable("mytable");
    +HTable table = new HTable(conf, "mytable");
     // combine row count and kv count for region
    -Map> results = table.coprocessorExec(
    -    RowCountProtocol.class,
    -    null, null,
    -    new Batch.Call>() {
    -        public Pair call(RowCountProtocol counter) {
    -          return new Pair(counter.getRowCount(), counter.getKeyValueCount());
    -        }
    -    });
    -
    +final ExampleProtos.CountRequest request = ExampleProtos.CountRequest.getDefaultInstance(); +Map results = table.coprocessorService( + ExampleProtos.RowCountService.class, // the protocol interface we're invoking + null, null, // start and end row keys + new Batch.Call>() { + public Long call(ExampleProtos.RowCountService counter) throws IOException { + BlockingRpcCallback rowCallback = + new BlockingRpcCallback(); + counter.getRowCount(null, request, rowCallback); -

    -Similarly, you could average the number of key-values per row for each region: -

    + BlockingRpcCallback kvCallback = + new BlockingRpcCallback(); + counter.getKeyValueCount(null, request, kvCallback); -
    -
    -Map results = table.coprocessorExec(
    -    RowCountProtocol.class,
    -    null, null,
    -    new Batch.Call() {
    -        public Double call(RowCountProtocol counter) {
    -          return ((double)counter.getKeyValueCount()) / ((double)counter.getRowCount());
    -        }
    -    });
    +         ExampleProtos.CountResponse rowResponse = rowCallback.get();
    +         ExampleProtos.CountResponse kvResponse = kvCallback.get();
    +         return new Pair(rowResponse.hasCount() ? rowResponse.getCount() : 0,
    +             kvResponse.hasCount() ? kvResponse.getCount() : 0);
    +    }
    +});
     
    */ package org.apache.hadoop.hbase.client.coprocessor; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java index 1d8a705cf40..7a91227e70f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.coprocessor; +import com.google.protobuf.Service; +import com.google.protobuf.ServiceException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; @@ -34,6 +36,7 @@ import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.client.*; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.ipc.CoprocessorProtocol; +import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.SortedCopyOnWriteSet; import org.apache.hadoop.hbase.util.VersionInfo; @@ -250,6 +253,11 @@ public abstract class CoprocessorHost { */ public E loadInstance(Class implClass, int priority, Configuration conf) throws IOException { + if (!Coprocessor.class.isAssignableFrom(implClass)) { + throw new IOException("Configured class " + implClass.getName() + " must implement " + + Coprocessor.class.getName() + " interface "); + } + // create the instance Coprocessor impl; Object o = null; @@ -435,7 +443,7 @@ public abstract class CoprocessorHost { byte[] qualifier, long amount, boolean writeToWAL) throws IOException { return table.incrementColumnValue(row, family, qualifier, amount, - writeToWAL); + writeToWAL); } @Override @@ -536,6 +544,25 @@ public abstract class CoprocessorHost { return table.coprocessorProxy(protocol, row); } + @Override + public CoprocessorRpcChannel coprocessorService(byte[] row) { + return table.coprocessorService(row); + } + + @Override + public Map coprocessorService(Class service, + byte[] startKey, byte[] endKey, Batch.Call callable) + throws ServiceException, Throwable { + return table.coprocessorService(service, startKey, endKey, callable); + } + + @Override + public void coprocessorService(Class service, + byte[] startKey, byte[] endKey, Batch.Call callable, Batch.Callback callback) + throws ServiceException, Throwable { + table.coprocessorService(service, startKey, endKey, callable, callback); + } + @Override public void mutateRow(RowMutations rm) throws IOException { table.mutateRow(rm); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorService.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorService.java new file mode 100644 index 00000000000..128a5b947c7 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorService.java @@ -0,0 +1,33 @@ +/* + * 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.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * Coprocessor endpoints providing protobuf services should implement this + * interface and return the {@link Service} instance via {@link #getService()}. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface CoprocessorService { + public Service getService(); +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java new file mode 100644 index 00000000000..465b4e07fa6 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java @@ -0,0 +1,164 @@ +/* + * 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.example; + +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; +import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.coprocessor.CoprocessorException; +import org.apache.hadoop.hbase.coprocessor.CoprocessorService; +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.protobuf.ResponseConverter; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.util.Bytes; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Sample coprocessor endpoint exposing a Service interface for counting rows and key values. + * + *

    + * For the protocol buffer definition of the RowCountService, see the source file located under + * hbase-server/src/main/protobuf/Examples.proto. + *

    + */ +public class RowCountEndpoint extends ExampleProtos.RowCountService + implements Coprocessor, CoprocessorService { + private RegionCoprocessorEnvironment env; + + public RowCountEndpoint() { + } + + /** + * Just returns a reference to this object, which implements the RowCounterService interface. + */ + @Override + public Service getService() { + return this; + } + + /** + * Returns a count of the rows in the region where this coprocessor is loaded. + */ + @Override + public void getRowCount(RpcController controller, ExampleProtos.CountRequest request, + RpcCallback done) { + Scan scan = new Scan(); + scan.setFilter(new FirstKeyOnlyFilter()); + ExampleProtos.CountResponse response = null; + InternalScanner scanner = null; + try { + scanner = env.getRegion().getScanner(scan); + List results = new ArrayList(); + boolean hasMore = false; + byte[] lastRow = null; + long count = 0; + do { + hasMore = scanner.next(results); + for (KeyValue kv : results) { + byte[] currentRow = kv.getRow(); + if (lastRow == null || !Bytes.equals(lastRow, currentRow)) { + lastRow = currentRow; + count++; + } + } + results.clear(); + } while (hasMore); + + response = ExampleProtos.CountResponse.newBuilder() + .setCount(count).build(); + } catch (IOException ioe) { + ResponseConverter.setControllerException(controller, ioe); + } finally { + if (scanner != null) { + try { + scanner.close(); + } catch (IOException ignored) {} + } + } + done.run(response); + } + + /** + * Returns a count of all KeyValues in the region where this coprocessor is loaded. + */ + @Override + public void getKeyValueCount(RpcController controller, ExampleProtos.CountRequest request, + RpcCallback done) { + ExampleProtos.CountResponse response = null; + InternalScanner scanner = null; + try { + scanner = env.getRegion().getScanner(new Scan()); + List results = new ArrayList(); + boolean hasMore = false; + long count = 0; + do { + hasMore = scanner.next(results); + for (KeyValue kv : results) { + count++; + } + results.clear(); + } while (hasMore); + + response = ExampleProtos.CountResponse.newBuilder() + .setCount(count).build(); + } catch (IOException ioe) { + ResponseConverter.setControllerException(controller, ioe); + } finally { + if (scanner != null) { + try { + scanner.close(); + } catch (IOException ignored) {} + } + } + done.run(response); + } + + /** + * Stores a reference to the coprocessor environment provided by the + * {@link org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost} from the region where this + * coprocessor is loaded. Since this is a coprocessor endpoint, it always expects to be loaded + * on a table region, so always expects this to be an instance of + * {@link RegionCoprocessorEnvironment}. + * @param env the environment provided by the coprocessor host + * @throws IOException if the provided environment is not an instance of + * {@code RegionCoprocessorEnvironment} + */ + @Override + public void start(CoprocessorEnvironment env) throws IOException { + if (env instanceof RegionCoprocessorEnvironment) { + this.env = (RegionCoprocessorEnvironment)env; + } else { + throw new CoprocessorException("Must be loaded on a table region!"); + } + } + + @Override + public void stop(CoprocessorEnvironment env) throws IOException { + // nothing to do + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/generated/ExampleProtos.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/generated/ExampleProtos.java new file mode 100644 index 00000000000..123bf3b5510 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/generated/ExampleProtos.java @@ -0,0 +1,1034 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: Examples.proto + +package org.apache.hadoop.hbase.coprocessor.example.generated; + +public final class ExampleProtos { + private ExampleProtos() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + } + public interface CountRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + } + public static final class CountRequest extends + com.google.protobuf.GeneratedMessage + implements CountRequestOrBuilder { + // Use CountRequest.newBuilder() to construct. + private CountRequest(Builder builder) { + super(builder); + } + private CountRequest(boolean noInit) {} + + private static final CountRequest defaultInstance; + public static CountRequest getDefaultInstance() { + return defaultInstance; + } + + public CountRequest getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountRequest_fieldAccessorTable; + } + + private void initFields() { + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest other = (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest) obj; + + boolean result = true; + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountRequest_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDescriptor(); + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest build() { + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest buildPartial() { + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest result = new org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest(this); + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest) { + return mergeFrom((org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest other) { + if (other == org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance()) return this; + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + } + } + } + + + // @@protoc_insertion_point(builder_scope:CountRequest) + } + + static { + defaultInstance = new CountRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:CountRequest) + } + + public interface CountResponseOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required int64 count = 1 [default = 0]; + boolean hasCount(); + long getCount(); + } + public static final class CountResponse extends + com.google.protobuf.GeneratedMessage + implements CountResponseOrBuilder { + // Use CountResponse.newBuilder() to construct. + private CountResponse(Builder builder) { + super(builder); + } + private CountResponse(boolean noInit) {} + + private static final CountResponse defaultInstance; + public static CountResponse getDefaultInstance() { + return defaultInstance; + } + + public CountResponse getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountResponse_fieldAccessorTable; + } + + private int bitField0_; + // required int64 count = 1 [default = 0]; + public static final int COUNT_FIELD_NUMBER = 1; + private long count_; + public boolean hasCount() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getCount() { + return count_; + } + + private void initFields() { + count_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasCount()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeInt64(1, count_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(1, count_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse other = (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse) obj; + + boolean result = true; + result = result && (hasCount() == other.hasCount()); + if (hasCount()) { + result = result && (getCount() + == other.getCount()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasCount()) { + hash = (37 * hash) + COUNT_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getCount()); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountResponse_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + count_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDescriptor(); + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse build() { + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse buildPartial() { + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse result = new org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.count_ = count_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse) { + return mergeFrom((org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse other) { + if (other == org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance()) return this; + if (other.hasCount()) { + setCount(other.getCount()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasCount()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + count_ = input.readInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required int64 count = 1 [default = 0]; + private long count_ ; + public boolean hasCount() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getCount() { + return count_; + } + public Builder setCount(long value) { + bitField0_ |= 0x00000001; + count_ = value; + onChanged(); + return this; + } + public Builder clearCount() { + bitField0_ = (bitField0_ & ~0x00000001); + count_ = 0L; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:CountResponse) + } + + static { + defaultInstance = new CountResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:CountResponse) + } + + public static abstract class RowCountService + implements com.google.protobuf.Service { + protected RowCountService() {} + + public interface Interface { + public abstract void getRowCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, + com.google.protobuf.RpcCallback done); + + public abstract void getKeyValueCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, + com.google.protobuf.RpcCallback done); + + } + + public static com.google.protobuf.Service newReflectiveService( + final Interface impl) { + return new RowCountService() { + @java.lang.Override + public void getRowCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, + com.google.protobuf.RpcCallback done) { + impl.getRowCount(controller, request, done); + } + + @java.lang.Override + public void getKeyValueCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, + com.google.protobuf.RpcCallback done) { + impl.getKeyValueCount(controller, request, done); + } + + }; + } + + public static com.google.protobuf.BlockingService + newReflectiveBlockingService(final BlockingInterface impl) { + return new com.google.protobuf.BlockingService() { + public final com.google.protobuf.Descriptors.ServiceDescriptor + getDescriptorForType() { + return getDescriptor(); + } + + public final com.google.protobuf.Message callBlockingMethod( + com.google.protobuf.Descriptors.MethodDescriptor method, + com.google.protobuf.RpcController controller, + com.google.protobuf.Message request) + throws com.google.protobuf.ServiceException { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.callBlockingMethod() given method descriptor for " + + "wrong service type."); + } + switch(method.getIndex()) { + case 0: + return impl.getRowCount(controller, (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)request); + case 1: + return impl.getKeyValueCount(controller, (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)request); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getRequestPrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getRequestPrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance(); + case 1: + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getResponsePrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getResponsePrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(); + case 1: + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + }; + } + + public abstract void getRowCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, + com.google.protobuf.RpcCallback done); + + public abstract void getKeyValueCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, + com.google.protobuf.RpcCallback done); + + public static final + com.google.protobuf.Descriptors.ServiceDescriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.getDescriptor().getServices().get(0); + } + public final com.google.protobuf.Descriptors.ServiceDescriptor + getDescriptorForType() { + return getDescriptor(); + } + + public final void callMethod( + com.google.protobuf.Descriptors.MethodDescriptor method, + com.google.protobuf.RpcController controller, + com.google.protobuf.Message request, + com.google.protobuf.RpcCallback< + com.google.protobuf.Message> done) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.callMethod() given method descriptor for wrong " + + "service type."); + } + switch(method.getIndex()) { + case 0: + this.getRowCount(controller, (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 1: + this.getKeyValueCount(controller, (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getRequestPrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getRequestPrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance(); + case 1: + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getResponsePrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getResponsePrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(); + case 1: + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public static Stub newStub( + com.google.protobuf.RpcChannel channel) { + return new Stub(channel); + } + + public static final class Stub extends org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.RowCountService implements Interface { + private Stub(com.google.protobuf.RpcChannel channel) { + this.channel = channel; + } + + private final com.google.protobuf.RpcChannel channel; + + public com.google.protobuf.RpcChannel getChannel() { + return channel; + } + + public void getRowCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(0), + controller, + request, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.class, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance())); + } + + public void getKeyValueCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(1), + controller, + request, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.class, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance())); + } + } + + public static BlockingInterface newBlockingStub( + com.google.protobuf.BlockingRpcChannel channel) { + return new BlockingStub(channel); + } + + public interface BlockingInterface { + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse getRowCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request) + throws com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse getKeyValueCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request) + throws com.google.protobuf.ServiceException; + } + + private static final class BlockingStub implements BlockingInterface { + private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) { + this.channel = channel; + } + + private final com.google.protobuf.BlockingRpcChannel channel; + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse getRowCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(0), + controller, + request, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance()); + } + + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse getKeyValueCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(1), + controller, + request, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance()); + } + + } + } + + private static com.google.protobuf.Descriptors.Descriptor + internal_static_CountRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_CountRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_CountResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_CountResponse_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\016Examples.proto\"\016\n\014CountRequest\"!\n\rCoun" + + "tResponse\022\020\n\005count\030\001 \002(\003:\00102r\n\017RowCountS" + + "ervice\022,\n\013getRowCount\022\r.CountRequest\032\016.C" + + "ountResponse\0221\n\020getKeyValueCount\022\r.Count" + + "Request\032\016.CountResponseBN\n5org.apache.ha" + + "doop.hbase.coprocessor.example.generated" + + "B\rExampleProtosH\001\210\001\001\240\001\001" + }; + com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { + public com.google.protobuf.ExtensionRegistry assignDescriptors( + com.google.protobuf.Descriptors.FileDescriptor root) { + descriptor = root; + internal_static_CountRequest_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_CountRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_CountRequest_descriptor, + new java.lang.String[] { }, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.class, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.Builder.class); + internal_static_CountResponse_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_CountResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_CountResponse_descriptor, + new java.lang.String[] { "Count", }, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.class, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.Builder.class); + return null; + } + }; + com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }, assigner); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/package-info.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/package-info.java index 4e8aec020ae..b392584fba7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/package-info.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/package-info.java @@ -171,117 +171,37 @@ public class AccessControlCoprocessor extends BaseRegionObserverCoprocessor {

    Endpoint

    Coprocessor and RegionObserver provide certain hooks -for injecting user code running at each region. The user code will be triggerd +for injecting user code running at each region. The user code will be triggered by existing HTable and HBaseAdmin operations at the certain hook points.

    -Through Endpoint and dynamic RPC protocol, you can define your own -interface communicated between client and region server, -i.e., you can create a new method, specify passed parameters and return types -for this new method. -And the new Endpoint methods can be triggered by -calling client side dynamic RPC functions -- HTable.coprocessorExec(...) +Coprocessor Endpoints allow you to define your own dynamic RPC protocol to communicate +between clients and region servers, i.e., you can create a new method, specifying custom +request parameters and return types. RPC methods exposed by coprocessor Endpoints can be +triggered by calling client side dynamic RPC functions -- HTable.coprocessorService(...) .

    -To implement a Endpoint, you need to: +To implement an Endpoint, you need to:

      -
    • Extend CoprocessorProtocol: the interface defines -communication protocol for the new Endpoint, and will be -served as the RPC protocol between client and region server.
    • -
    • Extend both BaseEndpointCoprocessor abstract class, -and the above extended CoprocessorProtocol interface: -the actually implemented class running at region server.
    • +
    • Define a protocol buffer Service and supporting Message types for the RPC methods. + See the + protocol buffer guide + for more details on defining services.
    • +
    • Generate the Service and Message code using the protoc compiler
    • +
    • Implement the generated Service interface in your coprocessor class and implement the + CoprocessorService interface. The CoprocessorService.getService() + method should return a reference to the Endpoint's protocol buffer Service instance.

    -Here's an example of performing column aggregation at region server: -

    -
    -// A sample protocol for performing aggregation at regions.
    -public static interface ColumnAggregationProtocol
    -extends CoprocessorProtocol {
    -  // Perform aggregation for a given column at the region. The aggregation
    -  // will include all the rows inside the region. It can be extended to
    -  // allow passing start and end rows for a fine-grained aggregation.
    -  public int sum(byte[] family, byte[] qualifier) throws IOException;
    -}
    -// Aggregation implementation at a region.
    -public static class ColumnAggregationEndpoint extends BaseEndpointCoprocessor
    -implements ColumnAggregationProtocol {
    -  // @Override
    -  // Scan the region by the given family and qualifier. Return the aggregation
    -  // result.
    -  public int sum(byte[] family, byte[] qualifier)
    -  throws IOException {
    -    // aggregate at each region
    -    Scan scan = new Scan();
    -    scan.addColumn(family, qualifier);
    -    int sumResult = 0;
    -    // use an internal scanner to perform scanning.
    -    InternalScanner scanner = getEnvironment().getRegion().getScanner(scan);
    -    try {
    -      List<KeyValue> curVals = new ArrayList<KeyValue>();
    -      boolean done = false;
    -      do {
    -        curVals.clear();
    -        done = scanner.next(curVals);
    -        KeyValue kv = curVals.get(0);
    -        sumResult += Bytes.toInt(kv.getValue());
    -      } while (done);
    -    } finally {
    -      scanner.close();
    -    }
    -    return sumResult;
    -  }
    -}
    -
    -
    -

    -Client invocations are performed through HTable, -which has the following methods added by dynamic RPC protocol: +For a more detailed discussion of how to implement a coprocessor Endpoint, along with some sample +code, see the {@link org.apache.hadoop.hbase.client.coprocessor} package documentation. +

    -
    -
    -public <T extends CoprocessorProtocol> T coprocessorProxy(Class<T> protocol, Row row)
    -
    -public <T extends CoprocessorProtocol, R> void coprocessorExec(
    -    Class<T> protocol, List<? extends Row> rows,
    -    BatchCall<T,R> callable, BatchCallback<R> callback)
    -
    -public <T extends CoprocessorProtocol, R> void coprocessorExec(
    -    Class<T> protocol, RowRange range,
    -    BatchCall<T,R> callable, BatchCallback<R> callback)
    -
    -
    - -

    -Here is a client side example of invoking -ColumnAggregationEndpoint: -

    -
    -HTable table = new HTable(util.getConfiguration(), TEST_TABLE);
    -Scan scan;
    -Map<byte[], Integer> results;
    -
    -// scan: for all regions
    -scan = new Scan();
    -results = table.coprocessorExec(ColumnAggregationProtocol.class, scan,
    -    new BatchCall<ColumnAggregationProtocol,Integer>() {
    -      public Integer call(ColumnAggregationProtocol instance) throws IOException{
    -        return instance.sum(TEST_FAMILY, TEST_QUALIFIER);
    -      }
    -    });
    -int sumResult = 0;
    -int expectedResult = 0;
    -for (Map.Entry<byte[], Integer> e : results.entrySet()) {
    -  sumResult += e.getValue();
    -}
    -
    -
    -

    Coprocess loading

    +

    Coprocessor loading

    A customized coprocessor can be loaded by two different ways, by configuration, or by HTableDescriptor for a newly created table.

    -(Currently we don't really have an on demand coprocessor loading machanism for +(Currently we don't really have an on demand coprocessor loading mechanism for opened regions.)

    Load from configuration

    Whenever a region is opened, it will read coprocessor class names from @@ -294,7 +214,7 @@ default coprocessors. The classes must be included in the classpath already.
       <property>
         <name>hbase.coprocessor.region.classes</name>
    -    <value>org.apache.hadoop.hbase.coprocessor.AccessControllCoprocessor, org.apache.hadoop.hbase.coprocessor.ColumnAggregationProtocol</value>
    +    <value>org.apache.hadoop.hbase.coprocessor.AccessControlCoprocessor, org.apache.hadoop.hbase.coprocessor.ColumnAggregationProtocol</value>
         <description>A comma-separated list of Coprocessors that are loaded by
         default. For any override coprocessor method from RegionObservor or
         Coprocessor, these classes' implementation will be called
    diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcCallback.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcCallback.java
    new file mode 100644
    index 00000000000..322e67664be
    --- /dev/null
    +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/BlockingRpcCallback.java
    @@ -0,0 +1,73 @@
    +/*
    + * 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.ipc;
    +
    +import com.google.protobuf.RpcCallback;
    +import org.apache.hadoop.classification.InterfaceAudience;
    +import org.apache.hadoop.classification.InterfaceStability;
    +
    +import java.io.IOException;
    +import java.io.InterruptedIOException;
    +
    +/**
    + * Simple {@link RpcCallback} implementation providing a
    + * {@link java.util.concurrent.Future}-like {@link BlockingRpcCallback#get()} method, which
    + * will block util the instance's {@link BlockingRpcCallback#run(Object)} method has been called.
    + * {@code R} is the RPC response type that will be passed to the {@link #run(Object)} method.
    + */
    +@InterfaceAudience.Public
    +@InterfaceStability.Evolving
    +public class BlockingRpcCallback implements RpcCallback {
    +  private R result;
    +  private boolean resultSet = false;
    +
    +  /**
    +   * Called on completion of the RPC call with the response object, or {@code null} in the case of
    +   * an error.
    +   * @param parameter the response object or {@code null} if an error occurred
    +   */
    +  @Override
    +  public void run(R parameter) {
    +    synchronized (this) {
    +      result = parameter;
    +      resultSet = true;
    +      this.notify();
    +    }
    +  }
    +
    +  /**
    +   * Returns the parameter passed to {@link #run(Object)} or {@code null} if a null value was
    +   * passed.  When used asynchronously, this method will block until the {@link #run(Object)}
    +   * method has been called.
    +   * @return the response object or {@code null} if no response was passed
    +   */
    +  public synchronized R get() throws IOException {
    +    while (!resultSet) {
    +      try {
    +        this.wait();
    +      } catch (InterruptedException ie) {
    +        Thread.currentThread().interrupt();
    +        InterruptedIOException exception = new InterruptedIOException(ie.getMessage());
    +        exception.initCause(ie);
    +        throw exception;
    +      }
    +    }
    +    return result;
    +  }
    +}
    diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorProtocol.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorProtocol.java
    index cb93b69a0b7..fa95ec26378 100644
    --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorProtocol.java
    +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorProtocol.java
    @@ -35,9 +35,12 @@ import org.apache.hadoop.classification.InterfaceStability;
      *   
  • an array or {@code java.util.List} of one of the above
  • *
*

+ * @deprecated since 0.96. Use {@link org.apache.hadoop.hbase.coprocessor.CoprocessorService} + * instead. */ @InterfaceAudience.Public @InterfaceStability.Evolving +@Deprecated public interface CoprocessorProtocol extends VersionedProtocol { public static final long VERSION = 1L; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java new file mode 100644 index 00000000000..92e7ef35ef3 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java @@ -0,0 +1,132 @@ +/* + * 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.ipc; + +import com.google.protobuf.*; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.client.ServerCallable; +import org.apache.hadoop.hbase.client.coprocessor.Exec; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.ResponseConverter; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; +import org.apache.hadoop.hbase.util.Bytes; + +import java.io.IOException; + +import static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse; + +/** + * Provides clients with an RPC connection to call coprocessor endpoint {@link Service}s + * against a given table region. An instance of this class may be obtained + * by calling {@link org.apache.hadoop.hbase.client.HTable#coprocessorService(byte[])}, + * but should normally only be used in creating a new {@link Service} stub to call the endpoint + * methods. + * @see org.apache.hadoop.hbase.client.HTable#coprocessorService(byte[]) + */ +@InterfaceAudience.Private +public class CoprocessorRpcChannel implements RpcChannel, BlockingRpcChannel { + private static Log LOG = LogFactory.getLog(CoprocessorRpcChannel.class); + + private final HConnection connection; + private final byte[] table; + private final byte[] row; + private byte[] lastRegion; + + public CoprocessorRpcChannel(HConnection conn, byte[] table, byte[] row) { + this.connection = conn; + this.table = table; + this.row = row; + } + + @Override + public void callMethod(Descriptors.MethodDescriptor method, + RpcController controller, + Message request, Message responsePrototype, + RpcCallback callback) { + Message response = null; + try { + response = callExecService(method, request, responsePrototype); + } catch (IOException ioe) { + LOG.warn("Call failed on IOException", ioe); + ResponseConverter.setControllerException(controller, ioe); + } + if (callback != null) { + callback.run(response); + } + } + + @Override + public Message callBlockingMethod(Descriptors.MethodDescriptor method, + RpcController controller, + Message request, Message responsePrototype) + throws ServiceException { + try { + return callExecService(method, request, responsePrototype); + } catch (IOException ioe) { + throw new ServiceException("Error calling method "+method.getFullName(), ioe); + } + } + + private Message callExecService(Descriptors.MethodDescriptor method, + Message request, Message responsePrototype) + throws IOException { + if (LOG.isDebugEnabled()) { + LOG.debug("Call: "+method.getName()+", "+request.toString()); + } + + if (row == null) { + throw new IllegalArgumentException("Missing row property for remote region location"); + } + + final ClientProtos.CoprocessorServiceCall call = + ClientProtos.CoprocessorServiceCall.newBuilder() + .setRow(ByteString.copyFrom(row)) + .setServiceName(method.getService().getFullName()) + .setMethodName(method.getName()) + .setRequest(request.toByteString()).build(); + ServerCallable callable = + new ServerCallable(connection, table, row) { + public CoprocessorServiceResponse call() throws Exception { + byte[] regionName = location.getRegionInfo().getRegionName(); + return ProtobufUtil.execService(server, call, regionName); + } + }; + CoprocessorServiceResponse result = callable.withRetries(); + Message response = null; + if (result.getValue().hasValue()) { + response = responsePrototype.newBuilderForType() + .mergeFrom(result.getValue().getValue()).build(); + } else { + response = responsePrototype.getDefaultInstanceForType(); + } + lastRegion = result.getRegion().getValue().toByteArray(); + if (LOG.isTraceEnabled()) { + LOG.trace("Result is region=" + Bytes.toStringBinary(lastRegion) + ", value=" + response); + } + return response; + } + + public byte[] getLastRegion() { + return lastRegion; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ExecRPCInvoker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ExecRPCInvoker.java index 0e7a9cb950f..decd9bf9f96 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ExecRPCInvoker.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ExecRPCInvoker.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.util.Bytes; * {@link org.apache.hadoop.hbase.client.ServerCallable} instance). */ @InterfaceAudience.Private +@Deprecated public class ExecRPCInvoker implements InvocationHandler { // LOG is NOT in hbase subpackage intentionally so that the default HBase // DEBUG log level does NOT emit RPC-level logging. @@ -84,6 +85,8 @@ public class ExecRPCInvoker implements InvocationHandler { LOG.debug("Result is region="+ Bytes.toStringBinary(regionName) + ", value="+result.getValue()); return result.getValue(); + } else if (LOG.isDebugEnabled()) { + LOG.debug("Null row passed for call"); } return null; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcController.java new file mode 100644 index 00000000000..4c89a8c8859 --- /dev/null +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/ServerRpcController.java @@ -0,0 +1,124 @@ +/* + * 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.ipc; + +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; +import org.apache.hadoop.util.StringUtils; + +import java.io.IOException; + +/** + * Used for server-side protobuf RPC service invocations. This handler allows + * invocation exceptions to easily be passed through to the RPC server from coprocessor + * {@link Service} implementations. + * + *

+ * When implementing {@link Service} defined methods, coprocessor endpoints can use the following + * pattern to pass exceptions back to the RPC client: + * + * public void myMethod(RpcController controller, MyRequest request, RpcCallback done) { + * MyResponse response = null; + * try { + * // do processing + * response = MyResponse.getDefaultInstance(); // or use a new builder to populate the response + * } catch (IOException ioe) { + * // pass exception back up + * ResponseConverter.setControllerException(controller, ioe); + * } + * done.run(response); + * } + * + *

+ */ +public class ServerRpcController implements RpcController { + /** + * The exception thrown within + * {@link Service#callMethod(Descriptors.MethodDescriptor, RpcController, Message, RpcCallback)}, + * if any. + */ + // TODO: it would be good widen this to just Throwable, but IOException is what we allow now + private IOException serviceException; + private String errorMessage; + + @Override + public void reset() { + serviceException = null; + errorMessage = null; + } + + @Override + public boolean failed() { + return (failedOnException() || errorMessage != null); + } + + @Override + public String errorText() { + return errorMessage; + } + + @Override + public void startCancel() { + // not implemented + } + + @Override + public void setFailed(String message) { + errorMessage = message; + } + + @Override + public boolean isCanceled() { + return false; + } + + @Override + public void notifyOnCancel(RpcCallback objectRpcCallback) { + // not implemented + } + + /** + * Sets an exception to be communicated back to the {@link Service} client. + * @param ioe the exception encountered during execution of the service method + */ + public void setFailedOn(IOException ioe) { + serviceException = ioe; + setFailed(StringUtils.stringifyException(ioe)); + } + + /** + * Returns any exception thrown during service method invocation, or {@code null} if no exception + * was thrown. This can be used by clients to receive exceptions generated by RPC calls, even + * when {@link RpcCallback}s are used and no {@link com.google.protobuf.ServiceException} is + * declared. + */ + public IOException getFailedOn() { + return serviceException; + } + + /** + * Returns whether or not a server exception was generated in the prior RPC invocation. + */ + public boolean failedOnException() { + return serviceException != null; + } +} diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java index 16ad63d5b6f..64f16a1a1c7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java @@ -70,6 +70,7 @@ import org.apache.hadoop.hbase.filter.ByteArrayComparable; import org.apache.hadoop.hbase.io.HbaseObjectWritable; import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.ipc.CoprocessorProtocol; +import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionRequest; @@ -93,6 +94,9 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Column; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetRequest; @@ -120,6 +124,7 @@ import org.apache.hadoop.hbase.security.access.Permission; import org.apache.hadoop.hbase.security.access.TablePermission; import org.apache.hadoop.hbase.security.access.UserPermission; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Methods; import org.apache.hadoop.hbase.util.Pair; import com.google.common.collect.ArrayListMultimap; @@ -127,8 +132,12 @@ import com.google.common.collect.ListMultimap; import com.google.protobuf.ByteString; import com.google.protobuf.Message; +import com.google.protobuf.RpcChannel; +import com.google.protobuf.Service; import com.google.protobuf.ServiceException; +import static org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType.*; + /** * Protobufs utility. */ @@ -1306,6 +1315,26 @@ public final class ProtobufUtil { } } + public static CoprocessorServiceResponse execService(final ClientProtocol client, + final CoprocessorServiceCall call, final byte[] regionName) throws IOException { + CoprocessorServiceRequest request = CoprocessorServiceRequest.newBuilder() + .setCall(call).setRegion( + RequestConverter.buildRegionSpecifier(REGION_NAME, regionName)).build(); + try { + CoprocessorServiceResponse response = + client.execService(null, request); + return response; + } catch (ServiceException se) { + throw getRemoteException(se); + } + } + + public static T newServiceStub(Class service, RpcChannel channel) + throws Exception { + return (T)Methods.call(service, null, "newStub", + new Class[]{ RpcChannel.class }, new Object[]{ channel }); + } + // End helpers for Client // Start helpers for Admin @@ -1609,7 +1638,7 @@ public final class ProtobufUtil { /** * Convert a client Permission to a Permission proto * - * @param action the client Permission + * @param perm the client Permission * @return the protobuf Permission */ public static AccessControlProtos.Permission toPermission(Permission perm) { @@ -1650,7 +1679,7 @@ public final class ProtobufUtil { /** * Converts a Permission.Action proto to a client Permission.Action object. * - * @param proto the protobuf Action + * @param action the protobuf Action * @return the converted Action */ public static Permission.Action toPermissionAction( @@ -1789,4 +1818,21 @@ public final class ProtobufUtil { return perms; } + + /** + * Unwraps an exception from a protobuf service into the underlying (expected) IOException. + * This method will always throw an exception. + * @param se the {@code ServiceException} instance to convert into an {@code IOException} + */ + public static void toIOException(ServiceException se) throws IOException { + if (se == null) { + throw new NullPointerException("Null service exception passed!"); + } + + Throwable cause = se.getCause(); + if (cause != null && cause instanceof IOException) { + throw (IOException)cause; + } + throw new IOException(se); + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java index e135b9611c0..0ea88e4a499 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java @@ -21,10 +21,13 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import com.google.protobuf.RpcController; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.ipc.ServerRpcController; +import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetOnlineRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetServerInfoResponse; @@ -40,10 +43,13 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanR import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorResponse; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdResponse; import org.apache.hadoop.hbase.regionserver.RegionOpeningState; +import org.apache.hadoop.hbase.security.access.UserPermission; import org.apache.hadoop.util.StringUtils; import com.google.protobuf.ByteString; +import static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse; + /** * Helper utility to build protocol buffer responses, * or retrieve data from protocol buffer responses. @@ -117,6 +123,18 @@ public final class ResponseConverter { return builder.build(); } + /** + * Converts the permissions list into a protocol buffer UserPermissionsResponse + */ + public static UserPermissionsResponse buildUserPermissionsResponse( + final List permissions) { + UserPermissionsResponse.Builder builder = UserPermissionsResponse.newBuilder(); + for (UserPermission perm : permissions) { + builder.addPermission(ProtobufUtil.toUserPermission(perm)); + } + return builder.build(); + } + // End utilities for Client // Start utilities for Admin @@ -249,4 +267,19 @@ public final class ResponseConverter { return GetLastFlushedSequenceIdResponse.newBuilder().setLastFlushedSequenceId(seqId).build(); } + /** + * Stores an exception encountered during RPC invocation so it can be passed back + * through to the client. + * @param controller the controller instance provided by the client when calling the service + * @param ioe the exception encountered + */ + public static void setControllerException(RpcController controller, IOException ioe) { + if (controller != null) { + if (controller instanceof ServerRpcController) { + ((ServerRpcController)controller).setFailedOn(ioe); + } else { + controller.setFailed(StringUtils.stringifyException(ioe)); + } + } + } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AccessControlProtos.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AccessControlProtos.java index 296536706fd..b246e53cb9f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AccessControlProtos.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/AccessControlProtos.java @@ -2524,6 +2524,3824 @@ public final class AccessControlProtos { // @@protoc_insertion_point(class_scope:UserTablePermissions) } + public interface GrantRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .UserPermission permission = 1; + boolean hasPermission(); + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission getPermission(); + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder getPermissionOrBuilder(); + } + public static final class GrantRequest extends + com.google.protobuf.GeneratedMessage + implements GrantRequestOrBuilder { + // Use GrantRequest.newBuilder() to construct. + private GrantRequest(Builder builder) { + super(builder); + } + private GrantRequest(boolean noInit) {} + + private static final GrantRequest defaultInstance; + public static GrantRequest getDefaultInstance() { + return defaultInstance; + } + + public GrantRequest getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_GrantRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_GrantRequest_fieldAccessorTable; + } + + private int bitField0_; + // required .UserPermission permission = 1; + public static final int PERMISSION_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission permission_; + public boolean hasPermission() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission getPermission() { + return permission_; + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder getPermissionOrBuilder() { + return permission_; + } + + private void initFields() { + permission_ = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasPermission()) { + memoizedIsInitialized = 0; + return false; + } + if (!getPermission().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, permission_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, permission_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest other = (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest) obj; + + boolean result = true; + result = result && (hasPermission() == other.hasPermission()); + if (hasPermission()) { + result = result && getPermission() + .equals(other.getPermission()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasPermission()) { + hash = (37 * hash) + PERMISSION_FIELD_NUMBER; + hash = (53 * hash) + getPermission().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_GrantRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_GrantRequest_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getPermissionFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (permissionBuilder_ == null) { + permission_ = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.getDefaultInstance(); + } else { + permissionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest build() { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest result = new org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (permissionBuilder_ == null) { + result.permission_ = permission_; + } else { + result.permission_ = permissionBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest.getDefaultInstance()) return this; + if (other.hasPermission()) { + mergePermission(other.getPermission()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasPermission()) { + + return false; + } + if (!getPermission().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.newBuilder(); + if (hasPermission()) { + subBuilder.mergeFrom(getPermission()); + } + input.readMessage(subBuilder, extensionRegistry); + setPermission(subBuilder.buildPartial()); + break; + } + } + } + } + + private int bitField0_; + + // required .UserPermission permission = 1; + private org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission permission_ = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder> permissionBuilder_; + public boolean hasPermission() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission getPermission() { + if (permissionBuilder_ == null) { + return permission_; + } else { + return permissionBuilder_.getMessage(); + } + } + public Builder setPermission(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission value) { + if (permissionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + permission_ = value; + onChanged(); + } else { + permissionBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder setPermission( + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder builderForValue) { + if (permissionBuilder_ == null) { + permission_ = builderForValue.build(); + onChanged(); + } else { + permissionBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder mergePermission(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission value) { + if (permissionBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + permission_ != org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.getDefaultInstance()) { + permission_ = + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.newBuilder(permission_).mergeFrom(value).buildPartial(); + } else { + permission_ = value; + } + onChanged(); + } else { + permissionBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder clearPermission() { + if (permissionBuilder_ == null) { + permission_ = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.getDefaultInstance(); + onChanged(); + } else { + permissionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder getPermissionBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getPermissionFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder getPermissionOrBuilder() { + if (permissionBuilder_ != null) { + return permissionBuilder_.getMessageOrBuilder(); + } else { + return permission_; + } + } + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder> + getPermissionFieldBuilder() { + if (permissionBuilder_ == null) { + permissionBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder>( + permission_, + getParentForChildren(), + isClean()); + permission_ = null; + } + return permissionBuilder_; + } + + // @@protoc_insertion_point(builder_scope:GrantRequest) + } + + static { + defaultInstance = new GrantRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:GrantRequest) + } + + public interface GrantResponseOrBuilder + extends com.google.protobuf.MessageOrBuilder { + } + public static final class GrantResponse extends + com.google.protobuf.GeneratedMessage + implements GrantResponseOrBuilder { + // Use GrantResponse.newBuilder() to construct. + private GrantResponse(Builder builder) { + super(builder); + } + private GrantResponse(boolean noInit) {} + + private static final GrantResponse defaultInstance; + public static GrantResponse getDefaultInstance() { + return defaultInstance; + } + + public GrantResponse getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_GrantResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_GrantResponse_fieldAccessorTable; + } + + private void initFields() { + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse other = (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse) obj; + + boolean result = true; + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_GrantResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_GrantResponse_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse build() { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse result = new org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse(this); + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse.getDefaultInstance()) return this; + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + } + } + } + + + // @@protoc_insertion_point(builder_scope:GrantResponse) + } + + static { + defaultInstance = new GrantResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:GrantResponse) + } + + public interface RevokeRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .UserPermission permission = 1; + boolean hasPermission(); + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission getPermission(); + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder getPermissionOrBuilder(); + } + public static final class RevokeRequest extends + com.google.protobuf.GeneratedMessage + implements RevokeRequestOrBuilder { + // Use RevokeRequest.newBuilder() to construct. + private RevokeRequest(Builder builder) { + super(builder); + } + private RevokeRequest(boolean noInit) {} + + private static final RevokeRequest defaultInstance; + public static RevokeRequest getDefaultInstance() { + return defaultInstance; + } + + public RevokeRequest getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_RevokeRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_RevokeRequest_fieldAccessorTable; + } + + private int bitField0_; + // required .UserPermission permission = 1; + public static final int PERMISSION_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission permission_; + public boolean hasPermission() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission getPermission() { + return permission_; + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder getPermissionOrBuilder() { + return permission_; + } + + private void initFields() { + permission_ = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasPermission()) { + memoizedIsInitialized = 0; + return false; + } + if (!getPermission().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, permission_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, permission_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest other = (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest) obj; + + boolean result = true; + result = result && (hasPermission() == other.hasPermission()); + if (hasPermission()) { + result = result && getPermission() + .equals(other.getPermission()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasPermission()) { + hash = (37 * hash) + PERMISSION_FIELD_NUMBER; + hash = (53 * hash) + getPermission().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_RevokeRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_RevokeRequest_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getPermissionFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (permissionBuilder_ == null) { + permission_ = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.getDefaultInstance(); + } else { + permissionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest build() { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest result = new org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (permissionBuilder_ == null) { + result.permission_ = permission_; + } else { + result.permission_ = permissionBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest.getDefaultInstance()) return this; + if (other.hasPermission()) { + mergePermission(other.getPermission()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasPermission()) { + + return false; + } + if (!getPermission().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.newBuilder(); + if (hasPermission()) { + subBuilder.mergeFrom(getPermission()); + } + input.readMessage(subBuilder, extensionRegistry); + setPermission(subBuilder.buildPartial()); + break; + } + } + } + } + + private int bitField0_; + + // required .UserPermission permission = 1; + private org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission permission_ = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder> permissionBuilder_; + public boolean hasPermission() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission getPermission() { + if (permissionBuilder_ == null) { + return permission_; + } else { + return permissionBuilder_.getMessage(); + } + } + public Builder setPermission(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission value) { + if (permissionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + permission_ = value; + onChanged(); + } else { + permissionBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder setPermission( + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder builderForValue) { + if (permissionBuilder_ == null) { + permission_ = builderForValue.build(); + onChanged(); + } else { + permissionBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder mergePermission(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission value) { + if (permissionBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + permission_ != org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.getDefaultInstance()) { + permission_ = + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.newBuilder(permission_).mergeFrom(value).buildPartial(); + } else { + permission_ = value; + } + onChanged(); + } else { + permissionBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder clearPermission() { + if (permissionBuilder_ == null) { + permission_ = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.getDefaultInstance(); + onChanged(); + } else { + permissionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder getPermissionBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getPermissionFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder getPermissionOrBuilder() { + if (permissionBuilder_ != null) { + return permissionBuilder_.getMessageOrBuilder(); + } else { + return permission_; + } + } + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder> + getPermissionFieldBuilder() { + if (permissionBuilder_ == null) { + permissionBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder>( + permission_, + getParentForChildren(), + isClean()); + permission_ = null; + } + return permissionBuilder_; + } + + // @@protoc_insertion_point(builder_scope:RevokeRequest) + } + + static { + defaultInstance = new RevokeRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:RevokeRequest) + } + + public interface RevokeResponseOrBuilder + extends com.google.protobuf.MessageOrBuilder { + } + public static final class RevokeResponse extends + com.google.protobuf.GeneratedMessage + implements RevokeResponseOrBuilder { + // Use RevokeResponse.newBuilder() to construct. + private RevokeResponse(Builder builder) { + super(builder); + } + private RevokeResponse(boolean noInit) {} + + private static final RevokeResponse defaultInstance; + public static RevokeResponse getDefaultInstance() { + return defaultInstance; + } + + public RevokeResponse getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_RevokeResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_RevokeResponse_fieldAccessorTable; + } + + private void initFields() { + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse other = (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse) obj; + + boolean result = true; + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_RevokeResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_RevokeResponse_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse build() { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse result = new org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse(this); + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse.getDefaultInstance()) return this; + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + } + } + } + + + // @@protoc_insertion_point(builder_scope:RevokeResponse) + } + + static { + defaultInstance = new RevokeResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:RevokeResponse) + } + + public interface UserPermissionsRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required bytes table = 1; + boolean hasTable(); + com.google.protobuf.ByteString getTable(); + } + public static final class UserPermissionsRequest extends + com.google.protobuf.GeneratedMessage + implements UserPermissionsRequestOrBuilder { + // Use UserPermissionsRequest.newBuilder() to construct. + private UserPermissionsRequest(Builder builder) { + super(builder); + } + private UserPermissionsRequest(boolean noInit) {} + + private static final UserPermissionsRequest defaultInstance; + public static UserPermissionsRequest getDefaultInstance() { + return defaultInstance; + } + + public UserPermissionsRequest getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_UserPermissionsRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_UserPermissionsRequest_fieldAccessorTable; + } + + private int bitField0_; + // required bytes table = 1; + public static final int TABLE_FIELD_NUMBER = 1; + private com.google.protobuf.ByteString table_; + public boolean hasTable() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public com.google.protobuf.ByteString getTable() { + return table_; + } + + private void initFields() { + table_ = com.google.protobuf.ByteString.EMPTY; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasTable()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, table_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, table_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest other = (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest) obj; + + boolean result = true; + result = result && (hasTable() == other.hasTable()); + if (hasTable()) { + result = result && getTable() + .equals(other.getTable()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasTable()) { + hash = (37 * hash) + TABLE_FIELD_NUMBER; + hash = (53 * hash) + getTable().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_UserPermissionsRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_UserPermissionsRequest_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + table_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest build() { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest result = new org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.table_ = table_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest.getDefaultInstance()) return this; + if (other.hasTable()) { + setTable(other.getTable()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasTable()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + table_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required bytes table = 1; + private com.google.protobuf.ByteString table_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasTable() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public com.google.protobuf.ByteString getTable() { + return table_; + } + public Builder setTable(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + table_ = value; + onChanged(); + return this; + } + public Builder clearTable() { + bitField0_ = (bitField0_ & ~0x00000001); + table_ = getDefaultInstance().getTable(); + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:UserPermissionsRequest) + } + + static { + defaultInstance = new UserPermissionsRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:UserPermissionsRequest) + } + + public interface UserPermissionsResponseOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // repeated .UserPermission permission = 1; + java.util.List + getPermissionList(); + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission getPermission(int index); + int getPermissionCount(); + java.util.List + getPermissionOrBuilderList(); + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder getPermissionOrBuilder( + int index); + } + public static final class UserPermissionsResponse extends + com.google.protobuf.GeneratedMessage + implements UserPermissionsResponseOrBuilder { + // Use UserPermissionsResponse.newBuilder() to construct. + private UserPermissionsResponse(Builder builder) { + super(builder); + } + private UserPermissionsResponse(boolean noInit) {} + + private static final UserPermissionsResponse defaultInstance; + public static UserPermissionsResponse getDefaultInstance() { + return defaultInstance; + } + + public UserPermissionsResponse getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_UserPermissionsResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_UserPermissionsResponse_fieldAccessorTable; + } + + // repeated .UserPermission permission = 1; + public static final int PERMISSION_FIELD_NUMBER = 1; + private java.util.List permission_; + public java.util.List getPermissionList() { + return permission_; + } + public java.util.List + getPermissionOrBuilderList() { + return permission_; + } + public int getPermissionCount() { + return permission_.size(); + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission getPermission(int index) { + return permission_.get(index); + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder getPermissionOrBuilder( + int index) { + return permission_.get(index); + } + + private void initFields() { + permission_ = java.util.Collections.emptyList(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + for (int i = 0; i < getPermissionCount(); i++) { + if (!getPermission(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + for (int i = 0; i < permission_.size(); i++) { + output.writeMessage(1, permission_.get(i)); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + for (int i = 0; i < permission_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, permission_.get(i)); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse other = (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse) obj; + + boolean result = true; + result = result && getPermissionList() + .equals(other.getPermissionList()); + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (getPermissionCount() > 0) { + hash = (37 * hash) + PERMISSION_FIELD_NUMBER; + hash = (53 * hash) + getPermissionList().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_UserPermissionsResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_UserPermissionsResponse_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getPermissionFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (permissionBuilder_ == null) { + permission_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + } else { + permissionBuilder_.clear(); + } + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse build() { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse result = new org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse(this); + int from_bitField0_ = bitField0_; + if (permissionBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + permission_ = java.util.Collections.unmodifiableList(permission_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.permission_ = permission_; + } else { + result.permission_ = permissionBuilder_.build(); + } + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse.getDefaultInstance()) return this; + if (permissionBuilder_ == null) { + if (!other.permission_.isEmpty()) { + if (permission_.isEmpty()) { + permission_ = other.permission_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensurePermissionIsMutable(); + permission_.addAll(other.permission_); + } + onChanged(); + } + } else { + if (!other.permission_.isEmpty()) { + if (permissionBuilder_.isEmpty()) { + permissionBuilder_.dispose(); + permissionBuilder_ = null; + permission_ = other.permission_; + bitField0_ = (bitField0_ & ~0x00000001); + permissionBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getPermissionFieldBuilder() : null; + } else { + permissionBuilder_.addAllMessages(other.permission_); + } + } + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + for (int i = 0; i < getPermissionCount(); i++) { + if (!getPermission(i).isInitialized()) { + + return false; + } + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addPermission(subBuilder.buildPartial()); + break; + } + } + } + } + + private int bitField0_; + + // repeated .UserPermission permission = 1; + private java.util.List permission_ = + java.util.Collections.emptyList(); + private void ensurePermissionIsMutable() { + if (!((bitField0_ & 0x00000001) == 0x00000001)) { + permission_ = new java.util.ArrayList(permission_); + bitField0_ |= 0x00000001; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder> permissionBuilder_; + + public java.util.List getPermissionList() { + if (permissionBuilder_ == null) { + return java.util.Collections.unmodifiableList(permission_); + } else { + return permissionBuilder_.getMessageList(); + } + } + public int getPermissionCount() { + if (permissionBuilder_ == null) { + return permission_.size(); + } else { + return permissionBuilder_.getCount(); + } + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission getPermission(int index) { + if (permissionBuilder_ == null) { + return permission_.get(index); + } else { + return permissionBuilder_.getMessage(index); + } + } + public Builder setPermission( + int index, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission value) { + if (permissionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensurePermissionIsMutable(); + permission_.set(index, value); + onChanged(); + } else { + permissionBuilder_.setMessage(index, value); + } + return this; + } + public Builder setPermission( + int index, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder builderForValue) { + if (permissionBuilder_ == null) { + ensurePermissionIsMutable(); + permission_.set(index, builderForValue.build()); + onChanged(); + } else { + permissionBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + public Builder addPermission(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission value) { + if (permissionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensurePermissionIsMutable(); + permission_.add(value); + onChanged(); + } else { + permissionBuilder_.addMessage(value); + } + return this; + } + public Builder addPermission( + int index, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission value) { + if (permissionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensurePermissionIsMutable(); + permission_.add(index, value); + onChanged(); + } else { + permissionBuilder_.addMessage(index, value); + } + return this; + } + public Builder addPermission( + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder builderForValue) { + if (permissionBuilder_ == null) { + ensurePermissionIsMutable(); + permission_.add(builderForValue.build()); + onChanged(); + } else { + permissionBuilder_.addMessage(builderForValue.build()); + } + return this; + } + public Builder addPermission( + int index, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder builderForValue) { + if (permissionBuilder_ == null) { + ensurePermissionIsMutable(); + permission_.add(index, builderForValue.build()); + onChanged(); + } else { + permissionBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + public Builder addAllPermission( + java.lang.Iterable values) { + if (permissionBuilder_ == null) { + ensurePermissionIsMutable(); + super.addAll(values, permission_); + onChanged(); + } else { + permissionBuilder_.addAllMessages(values); + } + return this; + } + public Builder clearPermission() { + if (permissionBuilder_ == null) { + permission_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + } else { + permissionBuilder_.clear(); + } + return this; + } + public Builder removePermission(int index) { + if (permissionBuilder_ == null) { + ensurePermissionIsMutable(); + permission_.remove(index); + onChanged(); + } else { + permissionBuilder_.remove(index); + } + return this; + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder getPermissionBuilder( + int index) { + return getPermissionFieldBuilder().getBuilder(index); + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder getPermissionOrBuilder( + int index) { + if (permissionBuilder_ == null) { + return permission_.get(index); } else { + return permissionBuilder_.getMessageOrBuilder(index); + } + } + public java.util.List + getPermissionOrBuilderList() { + if (permissionBuilder_ != null) { + return permissionBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(permission_); + } + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder addPermissionBuilder() { + return getPermissionFieldBuilder().addBuilder( + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.getDefaultInstance()); + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder addPermissionBuilder( + int index) { + return getPermissionFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.getDefaultInstance()); + } + public java.util.List + getPermissionBuilderList() { + return getPermissionFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder> + getPermissionFieldBuilder() { + if (permissionBuilder_ == null) { + permissionBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermission.Builder, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionOrBuilder>( + permission_, + ((bitField0_ & 0x00000001) == 0x00000001), + getParentForChildren(), + isClean()); + permission_ = null; + } + return permissionBuilder_; + } + + // @@protoc_insertion_point(builder_scope:UserPermissionsResponse) + } + + static { + defaultInstance = new UserPermissionsResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:UserPermissionsResponse) + } + + public interface CheckPermissionsRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // repeated .Permission permission = 1; + java.util.List + getPermissionList(); + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission getPermission(int index); + int getPermissionCount(); + java.util.List + getPermissionOrBuilderList(); + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.PermissionOrBuilder getPermissionOrBuilder( + int index); + } + public static final class CheckPermissionsRequest extends + com.google.protobuf.GeneratedMessage + implements CheckPermissionsRequestOrBuilder { + // Use CheckPermissionsRequest.newBuilder() to construct. + private CheckPermissionsRequest(Builder builder) { + super(builder); + } + private CheckPermissionsRequest(boolean noInit) {} + + private static final CheckPermissionsRequest defaultInstance; + public static CheckPermissionsRequest getDefaultInstance() { + return defaultInstance; + } + + public CheckPermissionsRequest getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_CheckPermissionsRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_CheckPermissionsRequest_fieldAccessorTable; + } + + // repeated .Permission permission = 1; + public static final int PERMISSION_FIELD_NUMBER = 1; + private java.util.List permission_; + public java.util.List getPermissionList() { + return permission_; + } + public java.util.List + getPermissionOrBuilderList() { + return permission_; + } + public int getPermissionCount() { + return permission_.size(); + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission getPermission(int index) { + return permission_.get(index); + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.PermissionOrBuilder getPermissionOrBuilder( + int index) { + return permission_.get(index); + } + + private void initFields() { + permission_ = java.util.Collections.emptyList(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + for (int i = 0; i < permission_.size(); i++) { + output.writeMessage(1, permission_.get(i)); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + for (int i = 0; i < permission_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, permission_.get(i)); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest other = (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest) obj; + + boolean result = true; + result = result && getPermissionList() + .equals(other.getPermissionList()); + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (getPermissionCount() > 0) { + hash = (37 * hash) + PERMISSION_FIELD_NUMBER; + hash = (53 * hash) + getPermissionList().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_CheckPermissionsRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_CheckPermissionsRequest_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getPermissionFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (permissionBuilder_ == null) { + permission_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + } else { + permissionBuilder_.clear(); + } + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest build() { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest result = new org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest(this); + int from_bitField0_ = bitField0_; + if (permissionBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { + permission_ = java.util.Collections.unmodifiableList(permission_); + bitField0_ = (bitField0_ & ~0x00000001); + } + result.permission_ = permission_; + } else { + result.permission_ = permissionBuilder_.build(); + } + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest.getDefaultInstance()) return this; + if (permissionBuilder_ == null) { + if (!other.permission_.isEmpty()) { + if (permission_.isEmpty()) { + permission_ = other.permission_; + bitField0_ = (bitField0_ & ~0x00000001); + } else { + ensurePermissionIsMutable(); + permission_.addAll(other.permission_); + } + onChanged(); + } + } else { + if (!other.permission_.isEmpty()) { + if (permissionBuilder_.isEmpty()) { + permissionBuilder_.dispose(); + permissionBuilder_ = null; + permission_ = other.permission_; + bitField0_ = (bitField0_ & ~0x00000001); + permissionBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getPermissionFieldBuilder() : null; + } else { + permissionBuilder_.addAllMessages(other.permission_); + } + } + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission.newBuilder(); + input.readMessage(subBuilder, extensionRegistry); + addPermission(subBuilder.buildPartial()); + break; + } + } + } + } + + private int bitField0_; + + // repeated .Permission permission = 1; + private java.util.List permission_ = + java.util.Collections.emptyList(); + private void ensurePermissionIsMutable() { + if (!((bitField0_ & 0x00000001) == 0x00000001)) { + permission_ = new java.util.ArrayList(permission_); + bitField0_ |= 0x00000001; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission.Builder, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.PermissionOrBuilder> permissionBuilder_; + + public java.util.List getPermissionList() { + if (permissionBuilder_ == null) { + return java.util.Collections.unmodifiableList(permission_); + } else { + return permissionBuilder_.getMessageList(); + } + } + public int getPermissionCount() { + if (permissionBuilder_ == null) { + return permission_.size(); + } else { + return permissionBuilder_.getCount(); + } + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission getPermission(int index) { + if (permissionBuilder_ == null) { + return permission_.get(index); + } else { + return permissionBuilder_.getMessage(index); + } + } + public Builder setPermission( + int index, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission value) { + if (permissionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensurePermissionIsMutable(); + permission_.set(index, value); + onChanged(); + } else { + permissionBuilder_.setMessage(index, value); + } + return this; + } + public Builder setPermission( + int index, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission.Builder builderForValue) { + if (permissionBuilder_ == null) { + ensurePermissionIsMutable(); + permission_.set(index, builderForValue.build()); + onChanged(); + } else { + permissionBuilder_.setMessage(index, builderForValue.build()); + } + return this; + } + public Builder addPermission(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission value) { + if (permissionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensurePermissionIsMutable(); + permission_.add(value); + onChanged(); + } else { + permissionBuilder_.addMessage(value); + } + return this; + } + public Builder addPermission( + int index, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission value) { + if (permissionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensurePermissionIsMutable(); + permission_.add(index, value); + onChanged(); + } else { + permissionBuilder_.addMessage(index, value); + } + return this; + } + public Builder addPermission( + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission.Builder builderForValue) { + if (permissionBuilder_ == null) { + ensurePermissionIsMutable(); + permission_.add(builderForValue.build()); + onChanged(); + } else { + permissionBuilder_.addMessage(builderForValue.build()); + } + return this; + } + public Builder addPermission( + int index, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission.Builder builderForValue) { + if (permissionBuilder_ == null) { + ensurePermissionIsMutable(); + permission_.add(index, builderForValue.build()); + onChanged(); + } else { + permissionBuilder_.addMessage(index, builderForValue.build()); + } + return this; + } + public Builder addAllPermission( + java.lang.Iterable values) { + if (permissionBuilder_ == null) { + ensurePermissionIsMutable(); + super.addAll(values, permission_); + onChanged(); + } else { + permissionBuilder_.addAllMessages(values); + } + return this; + } + public Builder clearPermission() { + if (permissionBuilder_ == null) { + permission_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000001); + onChanged(); + } else { + permissionBuilder_.clear(); + } + return this; + } + public Builder removePermission(int index) { + if (permissionBuilder_ == null) { + ensurePermissionIsMutable(); + permission_.remove(index); + onChanged(); + } else { + permissionBuilder_.remove(index); + } + return this; + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission.Builder getPermissionBuilder( + int index) { + return getPermissionFieldBuilder().getBuilder(index); + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.PermissionOrBuilder getPermissionOrBuilder( + int index) { + if (permissionBuilder_ == null) { + return permission_.get(index); } else { + return permissionBuilder_.getMessageOrBuilder(index); + } + } + public java.util.List + getPermissionOrBuilderList() { + if (permissionBuilder_ != null) { + return permissionBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(permission_); + } + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission.Builder addPermissionBuilder() { + return getPermissionFieldBuilder().addBuilder( + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission.getDefaultInstance()); + } + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission.Builder addPermissionBuilder( + int index) { + return getPermissionFieldBuilder().addBuilder( + index, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission.getDefaultInstance()); + } + public java.util.List + getPermissionBuilderList() { + return getPermissionFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission.Builder, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.PermissionOrBuilder> + getPermissionFieldBuilder() { + if (permissionBuilder_ == null) { + permissionBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.Permission.Builder, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.PermissionOrBuilder>( + permission_, + ((bitField0_ & 0x00000001) == 0x00000001), + getParentForChildren(), + isClean()); + permission_ = null; + } + return permissionBuilder_; + } + + // @@protoc_insertion_point(builder_scope:CheckPermissionsRequest) + } + + static { + defaultInstance = new CheckPermissionsRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:CheckPermissionsRequest) + } + + public interface CheckPermissionsResponseOrBuilder + extends com.google.protobuf.MessageOrBuilder { + } + public static final class CheckPermissionsResponse extends + com.google.protobuf.GeneratedMessage + implements CheckPermissionsResponseOrBuilder { + // Use CheckPermissionsResponse.newBuilder() to construct. + private CheckPermissionsResponse(Builder builder) { + super(builder); + } + private CheckPermissionsResponse(boolean noInit) {} + + private static final CheckPermissionsResponse defaultInstance; + public static CheckPermissionsResponse getDefaultInstance() { + return defaultInstance; + } + + public CheckPermissionsResponse getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_CheckPermissionsResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_CheckPermissionsResponse_fieldAccessorTable; + } + + private void initFields() { + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse other = (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse) obj; + + boolean result = true; + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_CheckPermissionsResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.internal_static_CheckPermissionsResponse_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse build() { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse result = new org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse(this); + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse.getDefaultInstance()) return this; + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + } + } + } + + + // @@protoc_insertion_point(builder_scope:CheckPermissionsResponse) + } + + static { + defaultInstance = new CheckPermissionsResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:CheckPermissionsResponse) + } + + public static abstract class AccessControlService + implements com.google.protobuf.Service { + protected AccessControlService() {} + + public interface Interface { + public abstract void grant( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest request, + com.google.protobuf.RpcCallback done); + + public abstract void revoke( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest request, + com.google.protobuf.RpcCallback done); + + public abstract void getUserPermissions( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest request, + com.google.protobuf.RpcCallback done); + + public abstract void checkPermissions( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest request, + com.google.protobuf.RpcCallback done); + + } + + public static com.google.protobuf.Service newReflectiveService( + final Interface impl) { + return new AccessControlService() { + @java.lang.Override + public void grant( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest request, + com.google.protobuf.RpcCallback done) { + impl.grant(controller, request, done); + } + + @java.lang.Override + public void revoke( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest request, + com.google.protobuf.RpcCallback done) { + impl.revoke(controller, request, done); + } + + @java.lang.Override + public void getUserPermissions( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest request, + com.google.protobuf.RpcCallback done) { + impl.getUserPermissions(controller, request, done); + } + + @java.lang.Override + public void checkPermissions( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest request, + com.google.protobuf.RpcCallback done) { + impl.checkPermissions(controller, request, done); + } + + }; + } + + public static com.google.protobuf.BlockingService + newReflectiveBlockingService(final BlockingInterface impl) { + return new com.google.protobuf.BlockingService() { + public final com.google.protobuf.Descriptors.ServiceDescriptor + getDescriptorForType() { + return getDescriptor(); + } + + public final com.google.protobuf.Message callBlockingMethod( + com.google.protobuf.Descriptors.MethodDescriptor method, + com.google.protobuf.RpcController controller, + com.google.protobuf.Message request) + throws com.google.protobuf.ServiceException { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.callBlockingMethod() given method descriptor for " + + "wrong service type."); + } + switch(method.getIndex()) { + case 0: + return impl.grant(controller, (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest)request); + case 1: + return impl.revoke(controller, (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest)request); + case 2: + return impl.getUserPermissions(controller, (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest)request); + case 3: + return impl.checkPermissions(controller, (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest)request); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getRequestPrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getRequestPrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest.getDefaultInstance(); + case 1: + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest.getDefaultInstance(); + case 2: + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest.getDefaultInstance(); + case 3: + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getResponsePrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getResponsePrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse.getDefaultInstance(); + case 1: + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse.getDefaultInstance(); + case 2: + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse.getDefaultInstance(); + case 3: + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + }; + } + + public abstract void grant( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest request, + com.google.protobuf.RpcCallback done); + + public abstract void revoke( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest request, + com.google.protobuf.RpcCallback done); + + public abstract void getUserPermissions( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest request, + com.google.protobuf.RpcCallback done); + + public abstract void checkPermissions( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest request, + com.google.protobuf.RpcCallback done); + + public static final + com.google.protobuf.Descriptors.ServiceDescriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.getDescriptor().getServices().get(0); + } + public final com.google.protobuf.Descriptors.ServiceDescriptor + getDescriptorForType() { + return getDescriptor(); + } + + public final void callMethod( + com.google.protobuf.Descriptors.MethodDescriptor method, + com.google.protobuf.RpcController controller, + com.google.protobuf.Message request, + com.google.protobuf.RpcCallback< + com.google.protobuf.Message> done) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.callMethod() given method descriptor for wrong " + + "service type."); + } + switch(method.getIndex()) { + case 0: + this.grant(controller, (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 1: + this.revoke(controller, (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 2: + this.getUserPermissions(controller, (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 3: + this.checkPermissions(controller, (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getRequestPrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getRequestPrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest.getDefaultInstance(); + case 1: + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest.getDefaultInstance(); + case 2: + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest.getDefaultInstance(); + case 3: + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getResponsePrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getResponsePrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse.getDefaultInstance(); + case 1: + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse.getDefaultInstance(); + case 2: + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse.getDefaultInstance(); + case 3: + return org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public static Stub newStub( + com.google.protobuf.RpcChannel channel) { + return new Stub(channel); + } + + public static final class Stub extends org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService implements Interface { + private Stub(com.google.protobuf.RpcChannel channel) { + this.channel = channel; + } + + private final com.google.protobuf.RpcChannel channel; + + public com.google.protobuf.RpcChannel getChannel() { + return channel; + } + + public void grant( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(0), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse.class, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse.getDefaultInstance())); + } + + public void revoke( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(1), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse.class, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse.getDefaultInstance())); + } + + public void getUserPermissions( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(2), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse.class, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse.getDefaultInstance())); + } + + public void checkPermissions( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(3), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse.class, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse.getDefaultInstance())); + } + } + + public static BlockingInterface newBlockingStub( + com.google.protobuf.BlockingRpcChannel channel) { + return new BlockingStub(channel); + } + + public interface BlockingInterface { + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse grant( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest request) + throws com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse revoke( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest request) + throws com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse getUserPermissions( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest request) + throws com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse checkPermissions( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest request) + throws com.google.protobuf.ServiceException; + } + + private static final class BlockingStub implements BlockingInterface { + private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) { + this.channel = channel; + } + + private final com.google.protobuf.BlockingRpcChannel channel; + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse grant( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(0), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse.getDefaultInstance()); + } + + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse revoke( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(1), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse.getDefaultInstance()); + } + + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse getUserPermissions( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(2), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse.getDefaultInstance()); + } + + + public org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse checkPermissions( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(3), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse.getDefaultInstance()); + } + + } + } + private static com.google.protobuf.Descriptors.Descriptor internal_static_Permission_descriptor; private static @@ -2544,6 +6362,46 @@ public final class AccessControlProtos { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_UserTablePermissions_UserPermissions_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_GrantRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_GrantRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_GrantResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_GrantResponse_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_RevokeRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_RevokeRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_RevokeResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_RevokeResponse_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_UserPermissionsRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_UserPermissionsRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_UserPermissionsResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_UserPermissionsResponse_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_CheckPermissionsRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_CheckPermissionsRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_CheckPermissionsResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_CheckPermissionsResponse_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -2563,8 +6421,23 @@ public final class AccessControlProtos { "permissions\030\001 \003(\0132%.UserTablePermissions" + ".UserPermissions\032A\n\017UserPermissions\022\014\n\004u" + "ser\030\001 \002(\014\022 \n\013permissions\030\002 \003(\0132\013.Permiss", - "ionBI\n*org.apache.hadoop.hbase.protobuf." + - "generatedB\023AccessControlProtosH\001\210\001\001\240\001\001" + "ion\"3\n\014GrantRequest\022#\n\npermission\030\001 \002(\0132" + + "\017.UserPermission\"\017\n\rGrantResponse\"4\n\rRev" + + "okeRequest\022#\n\npermission\030\001 \002(\0132\017.UserPer" + + "mission\"\020\n\016RevokeResponse\"\'\n\026UserPermiss" + + "ionsRequest\022\r\n\005table\030\001 \002(\014\">\n\027UserPermis" + + "sionsResponse\022#\n\npermission\030\001 \003(\0132\017.User" + + "Permission\":\n\027CheckPermissionsRequest\022\037\n" + + "\npermission\030\001 \003(\0132\013.Permission\"\032\n\030CheckP" + + "ermissionsResponse2\373\001\n\024AccessControlServ" + + "ice\022&\n\005grant\022\r.GrantRequest\032\016.GrantRespo", + "nse\022)\n\006revoke\022\016.RevokeRequest\032\017.RevokeRe" + + "sponse\022G\n\022getUserPermissions\022\027.UserPermi" + + "ssionsRequest\032\030.UserPermissionsResponse\022" + + "G\n\020checkPermissions\022\030.CheckPermissionsRe" + + "quest\032\031.CheckPermissionsResponseBI\n*org." + + "apache.hadoop.hbase.protobuf.generatedB\023" + + "AccessControlProtosH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -2603,6 +6476,70 @@ public final class AccessControlProtos { new java.lang.String[] { "User", "Permissions", }, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserTablePermissions.UserPermissions.class, org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserTablePermissions.UserPermissions.Builder.class); + internal_static_GrantRequest_descriptor = + getDescriptor().getMessageTypes().get(3); + internal_static_GrantRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_GrantRequest_descriptor, + new java.lang.String[] { "Permission", }, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest.class, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantRequest.Builder.class); + internal_static_GrantResponse_descriptor = + getDescriptor().getMessageTypes().get(4); + internal_static_GrantResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_GrantResponse_descriptor, + new java.lang.String[] { }, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse.class, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.GrantResponse.Builder.class); + internal_static_RevokeRequest_descriptor = + getDescriptor().getMessageTypes().get(5); + internal_static_RevokeRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_RevokeRequest_descriptor, + new java.lang.String[] { "Permission", }, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest.class, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeRequest.Builder.class); + internal_static_RevokeResponse_descriptor = + getDescriptor().getMessageTypes().get(6); + internal_static_RevokeResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_RevokeResponse_descriptor, + new java.lang.String[] { }, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse.class, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.RevokeResponse.Builder.class); + internal_static_UserPermissionsRequest_descriptor = + getDescriptor().getMessageTypes().get(7); + internal_static_UserPermissionsRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_UserPermissionsRequest_descriptor, + new java.lang.String[] { "Table", }, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest.class, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsRequest.Builder.class); + internal_static_UserPermissionsResponse_descriptor = + getDescriptor().getMessageTypes().get(8); + internal_static_UserPermissionsResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_UserPermissionsResponse_descriptor, + new java.lang.String[] { "Permission", }, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse.class, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.UserPermissionsResponse.Builder.class); + internal_static_CheckPermissionsRequest_descriptor = + getDescriptor().getMessageTypes().get(9); + internal_static_CheckPermissionsRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_CheckPermissionsRequest_descriptor, + new java.lang.String[] { "Permission", }, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest.class, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest.Builder.class); + internal_static_CheckPermissionsResponse_descriptor = + getDescriptor().getMessageTypes().get(10); + internal_static_CheckPermissionsResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_CheckPermissionsResponse_descriptor, + new java.lang.String[] { }, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse.class, + org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsResponse.Builder.class); return null; } }; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java index 81f6cbea9f7..4a8c95fff03 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java @@ -17952,6 +17952,1969 @@ public final class ClientProtos { // @@protoc_insertion_point(class_scope:ExecCoprocessorResponse) } + public interface CoprocessorServiceCallOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required bytes row = 1; + boolean hasRow(); + com.google.protobuf.ByteString getRow(); + + // required string serviceName = 2; + boolean hasServiceName(); + String getServiceName(); + + // required string methodName = 3; + boolean hasMethodName(); + String getMethodName(); + + // required bytes request = 4; + boolean hasRequest(); + com.google.protobuf.ByteString getRequest(); + } + public static final class CoprocessorServiceCall extends + com.google.protobuf.GeneratedMessage + implements CoprocessorServiceCallOrBuilder { + // Use CoprocessorServiceCall.newBuilder() to construct. + private CoprocessorServiceCall(Builder builder) { + super(builder); + } + private CoprocessorServiceCall(boolean noInit) {} + + private static final CoprocessorServiceCall defaultInstance; + public static CoprocessorServiceCall getDefaultInstance() { + return defaultInstance; + } + + public CoprocessorServiceCall getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_CoprocessorServiceCall_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_CoprocessorServiceCall_fieldAccessorTable; + } + + private int bitField0_; + // required bytes row = 1; + public static final int ROW_FIELD_NUMBER = 1; + private com.google.protobuf.ByteString row_; + public boolean hasRow() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public com.google.protobuf.ByteString getRow() { + return row_; + } + + // required string serviceName = 2; + public static final int SERVICENAME_FIELD_NUMBER = 2; + private java.lang.Object serviceName_; + public boolean hasServiceName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getServiceName() { + java.lang.Object ref = serviceName_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + serviceName_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getServiceNameBytes() { + java.lang.Object ref = serviceName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + serviceName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // required string methodName = 3; + public static final int METHODNAME_FIELD_NUMBER = 3; + private java.lang.Object methodName_; + public boolean hasMethodName() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getMethodName() { + java.lang.Object ref = methodName_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + methodName_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getMethodNameBytes() { + java.lang.Object ref = methodName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + methodName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // required bytes request = 4; + public static final int REQUEST_FIELD_NUMBER = 4; + private com.google.protobuf.ByteString request_; + public boolean hasRequest() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public com.google.protobuf.ByteString getRequest() { + return request_; + } + + private void initFields() { + row_ = com.google.protobuf.ByteString.EMPTY; + serviceName_ = ""; + methodName_ = ""; + request_ = com.google.protobuf.ByteString.EMPTY; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRow()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasServiceName()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasMethodName()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasRequest()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, row_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getServiceNameBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getMethodNameBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(4, request_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, row_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, getServiceNameBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(3, getMethodNameBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(4, request_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall) obj; + + boolean result = true; + result = result && (hasRow() == other.hasRow()); + if (hasRow()) { + result = result && getRow() + .equals(other.getRow()); + } + result = result && (hasServiceName() == other.hasServiceName()); + if (hasServiceName()) { + result = result && getServiceName() + .equals(other.getServiceName()); + } + result = result && (hasMethodName() == other.hasMethodName()); + if (hasMethodName()) { + result = result && getMethodName() + .equals(other.getMethodName()); + } + result = result && (hasRequest() == other.hasRequest()); + if (hasRequest()) { + result = result && getRequest() + .equals(other.getRequest()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasRow()) { + hash = (37 * hash) + ROW_FIELD_NUMBER; + hash = (53 * hash) + getRow().hashCode(); + } + if (hasServiceName()) { + hash = (37 * hash) + SERVICENAME_FIELD_NUMBER; + hash = (53 * hash) + getServiceName().hashCode(); + } + if (hasMethodName()) { + hash = (37 * hash) + METHODNAME_FIELD_NUMBER; + hash = (53 * hash) + getMethodName().hashCode(); + } + if (hasRequest()) { + hash = (37 * hash) + REQUEST_FIELD_NUMBER; + hash = (53 * hash) + getRequest().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_CoprocessorServiceCall_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_CoprocessorServiceCall_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + row_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + serviceName_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + methodName_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + request_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall build() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.row_ = row_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.serviceName_ = serviceName_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.methodName_ = methodName_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.request_ = request_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.getDefaultInstance()) return this; + if (other.hasRow()) { + setRow(other.getRow()); + } + if (other.hasServiceName()) { + setServiceName(other.getServiceName()); + } + if (other.hasMethodName()) { + setMethodName(other.getMethodName()); + } + if (other.hasRequest()) { + setRequest(other.getRequest()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasRow()) { + + return false; + } + if (!hasServiceName()) { + + return false; + } + if (!hasMethodName()) { + + return false; + } + if (!hasRequest()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + row_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + serviceName_ = input.readBytes(); + break; + } + case 26: { + bitField0_ |= 0x00000004; + methodName_ = input.readBytes(); + break; + } + case 34: { + bitField0_ |= 0x00000008; + request_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required bytes row = 1; + private com.google.protobuf.ByteString row_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasRow() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public com.google.protobuf.ByteString getRow() { + return row_; + } + public Builder setRow(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + row_ = value; + onChanged(); + return this; + } + public Builder clearRow() { + bitField0_ = (bitField0_ & ~0x00000001); + row_ = getDefaultInstance().getRow(); + onChanged(); + return this; + } + + // required string serviceName = 2; + private java.lang.Object serviceName_ = ""; + public boolean hasServiceName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getServiceName() { + java.lang.Object ref = serviceName_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + serviceName_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setServiceName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + serviceName_ = value; + onChanged(); + return this; + } + public Builder clearServiceName() { + bitField0_ = (bitField0_ & ~0x00000002); + serviceName_ = getDefaultInstance().getServiceName(); + onChanged(); + return this; + } + void setServiceName(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000002; + serviceName_ = value; + onChanged(); + } + + // required string methodName = 3; + private java.lang.Object methodName_ = ""; + public boolean hasMethodName() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getMethodName() { + java.lang.Object ref = methodName_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + methodName_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setMethodName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + methodName_ = value; + onChanged(); + return this; + } + public Builder clearMethodName() { + bitField0_ = (bitField0_ & ~0x00000004); + methodName_ = getDefaultInstance().getMethodName(); + onChanged(); + return this; + } + void setMethodName(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000004; + methodName_ = value; + onChanged(); + } + + // required bytes request = 4; + private com.google.protobuf.ByteString request_ = com.google.protobuf.ByteString.EMPTY; + public boolean hasRequest() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public com.google.protobuf.ByteString getRequest() { + return request_; + } + public Builder setRequest(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + request_ = value; + onChanged(); + return this; + } + public Builder clearRequest() { + bitField0_ = (bitField0_ & ~0x00000008); + request_ = getDefaultInstance().getRequest(); + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:CoprocessorServiceCall) + } + + static { + defaultInstance = new CoprocessorServiceCall(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:CoprocessorServiceCall) + } + + public interface CoprocessorServiceRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .RegionSpecifier region = 1; + boolean hasRegion(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(); + + // required .CoprocessorServiceCall call = 2; + boolean hasCall(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall getCall(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder getCallOrBuilder(); + } + public static final class CoprocessorServiceRequest extends + com.google.protobuf.GeneratedMessage + implements CoprocessorServiceRequestOrBuilder { + // Use CoprocessorServiceRequest.newBuilder() to construct. + private CoprocessorServiceRequest(Builder builder) { + super(builder); + } + private CoprocessorServiceRequest(boolean noInit) {} + + private static final CoprocessorServiceRequest defaultInstance; + public static CoprocessorServiceRequest getDefaultInstance() { + return defaultInstance; + } + + public CoprocessorServiceRequest getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_CoprocessorServiceRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_CoprocessorServiceRequest_fieldAccessorTable; + } + + private int bitField0_; + // required .RegionSpecifier region = 1; + public static final int REGION_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier region_; + public boolean hasRegion() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegion() { + return region_; + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder() { + return region_; + } + + // required .CoprocessorServiceCall call = 2; + public static final int CALL_FIELD_NUMBER = 2; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall call_; + public boolean hasCall() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall getCall() { + return call_; + } + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder getCallOrBuilder() { + return call_; + } + + private void initFields() { + region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); + call_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRegion()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasCall()) { + memoizedIsInitialized = 0; + return false; + } + if (!getRegion().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + if (!getCall().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, region_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, call_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, region_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, call_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest) obj; + + boolean result = true; + result = result && (hasRegion() == other.hasRegion()); + if (hasRegion()) { + result = result && getRegion() + .equals(other.getRegion()); + } + result = result && (hasCall() == other.hasCall()); + if (hasCall()) { + result = result && getCall() + .equals(other.getCall()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasRegion()) { + hash = (37 * hash) + REGION_FIELD_NUMBER; + hash = (53 * hash) + getRegion().hashCode(); + } + if (hasCall()) { + hash = (37 * hash) + CALL_FIELD_NUMBER; + hash = (53 * hash) + getCall().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_CoprocessorServiceRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_CoprocessorServiceRequest_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getRegionFieldBuilder(); + getCallFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (regionBuilder_ == null) { + region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); + } else { + regionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + if (callBuilder_ == null) { + call_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.getDefaultInstance(); + } else { + callBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest build() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (regionBuilder_ == null) { + result.region_ = region_; + } else { + result.region_ = regionBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (callBuilder_ == null) { + result.call_ = call_; + } else { + result.call_ = callBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest.getDefaultInstance()) return this; + if (other.hasRegion()) { + mergeRegion(other.getRegion()); + } + if (other.hasCall()) { + mergeCall(other.getCall()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasRegion()) { + + return false; + } + if (!hasCall()) { + + return false; + } + if (!getRegion().isInitialized()) { + + return false; + } + if (!getCall().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.newBuilder(); + if (hasRegion()) { + subBuilder.mergeFrom(getRegion()); + } + input.readMessage(subBuilder, extensionRegistry); + setRegion(subBuilder.buildPartial()); + break; + } + case 18: { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.newBuilder(); + if (hasCall()) { + subBuilder.mergeFrom(getCall()); + } + input.readMessage(subBuilder, extensionRegistry); + setCall(subBuilder.buildPartial()); + break; + } + } + } + } + + private int bitField0_; + + // required .RegionSpecifier region = 1; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_; + public boolean hasRegion() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegion() { + if (regionBuilder_ == null) { + return region_; + } else { + return regionBuilder_.getMessage(); + } + } + public Builder setRegion(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier value) { + if (regionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + region_ = value; + onChanged(); + } else { + regionBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder setRegion( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) { + if (regionBuilder_ == null) { + region_ = builderForValue.build(); + onChanged(); + } else { + regionBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder mergeRegion(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier value) { + if (regionBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + region_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance()) { + region_ = + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.newBuilder(region_).mergeFrom(value).buildPartial(); + } else { + region_ = value; + } + onChanged(); + } else { + regionBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder clearRegion() { + if (regionBuilder_ == null) { + region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); + onChanged(); + } else { + regionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder getRegionBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getRegionFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder() { + if (regionBuilder_ != null) { + return regionBuilder_.getMessageOrBuilder(); + } else { + return region_; + } + } + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> + getRegionFieldBuilder() { + if (regionBuilder_ == null) { + regionBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>( + region_, + getParentForChildren(), + isClean()); + region_ = null; + } + return regionBuilder_; + } + + // required .CoprocessorServiceCall call = 2; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall call_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder> callBuilder_; + public boolean hasCall() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall getCall() { + if (callBuilder_ == null) { + return call_; + } else { + return callBuilder_.getMessage(); + } + } + public Builder setCall(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall value) { + if (callBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + call_ = value; + onChanged(); + } else { + callBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder setCall( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.Builder builderForValue) { + if (callBuilder_ == null) { + call_ = builderForValue.build(); + onChanged(); + } else { + callBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder mergeCall(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall value) { + if (callBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + call_ != org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.getDefaultInstance()) { + call_ = + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.newBuilder(call_).mergeFrom(value).buildPartial(); + } else { + call_ = value; + } + onChanged(); + } else { + callBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder clearCall() { + if (callBuilder_ == null) { + call_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.getDefaultInstance(); + onChanged(); + } else { + callBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.Builder getCallBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getCallFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder getCallOrBuilder() { + if (callBuilder_ != null) { + return callBuilder_.getMessageOrBuilder(); + } else { + return call_; + } + } + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder> + getCallFieldBuilder() { + if (callBuilder_ == null) { + callBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder>( + call_, + getParentForChildren(), + isClean()); + call_ = null; + } + return callBuilder_; + } + + // @@protoc_insertion_point(builder_scope:CoprocessorServiceRequest) + } + + static { + defaultInstance = new CoprocessorServiceRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:CoprocessorServiceRequest) + } + + public interface CoprocessorServiceResponseOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .RegionSpecifier region = 1; + boolean hasRegion(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegion(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder(); + + // required .NameBytesPair value = 2; + boolean hasValue(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getValue(); + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getValueOrBuilder(); + } + public static final class CoprocessorServiceResponse extends + com.google.protobuf.GeneratedMessage + implements CoprocessorServiceResponseOrBuilder { + // Use CoprocessorServiceResponse.newBuilder() to construct. + private CoprocessorServiceResponse(Builder builder) { + super(builder); + } + private CoprocessorServiceResponse(boolean noInit) {} + + private static final CoprocessorServiceResponse defaultInstance; + public static CoprocessorServiceResponse getDefaultInstance() { + return defaultInstance; + } + + public CoprocessorServiceResponse getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_CoprocessorServiceResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_CoprocessorServiceResponse_fieldAccessorTable; + } + + private int bitField0_; + // required .RegionSpecifier region = 1; + public static final int REGION_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier region_; + public boolean hasRegion() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegion() { + return region_; + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder() { + return region_; + } + + // required .NameBytesPair value = 2; + public static final int VALUE_FIELD_NUMBER = 2; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value_; + public boolean hasValue() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getValue() { + return value_; + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getValueOrBuilder() { + return value_; + } + + private void initFields() { + region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); + value_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRegion()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasValue()) { + memoizedIsInitialized = 0; + return false; + } + if (!getRegion().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + if (!getValue().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, region_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, value_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, region_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(2, value_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse) obj; + + boolean result = true; + result = result && (hasRegion() == other.hasRegion()); + if (hasRegion()) { + result = result && getRegion() + .equals(other.getRegion()); + } + result = result && (hasValue() == other.hasValue()); + if (hasValue()) { + result = result && getValue() + .equals(other.getValue()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasRegion()) { + hash = (37 * hash) + REGION_FIELD_NUMBER; + hash = (53 * hash) + getRegion().hashCode(); + } + if (hasValue()) { + hash = (37 * hash) + VALUE_FIELD_NUMBER; + hash = (53 * hash) + getValue().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_CoprocessorServiceResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_CoprocessorServiceResponse_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getRegionFieldBuilder(); + getValueFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (regionBuilder_ == null) { + region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); + } else { + regionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + if (valueBuilder_ == null) { + value_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); + } else { + valueBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse build() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (regionBuilder_ == null) { + result.region_ = region_; + } else { + result.region_ = regionBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (valueBuilder_ == null) { + result.value_ = value_; + } else { + result.value_ = valueBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance()) return this; + if (other.hasRegion()) { + mergeRegion(other.getRegion()); + } + if (other.hasValue()) { + mergeValue(other.getValue()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasRegion()) { + + return false; + } + if (!hasValue()) { + + return false; + } + if (!getRegion().isInitialized()) { + + return false; + } + if (!getValue().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.newBuilder(); + if (hasRegion()) { + subBuilder.mergeFrom(getRegion()); + } + input.readMessage(subBuilder, extensionRegistry); + setRegion(subBuilder.buildPartial()); + break; + } + case 18: { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.newBuilder(); + if (hasValue()) { + subBuilder.mergeFrom(getValue()); + } + input.readMessage(subBuilder, extensionRegistry); + setValue(subBuilder.buildPartial()); + break; + } + } + } + } + + private int bitField0_; + + // required .RegionSpecifier region = 1; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> regionBuilder_; + public boolean hasRegion() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier getRegion() { + if (regionBuilder_ == null) { + return region_; + } else { + return regionBuilder_.getMessage(); + } + } + public Builder setRegion(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier value) { + if (regionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + region_ = value; + onChanged(); + } else { + regionBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder setRegion( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder builderForValue) { + if (regionBuilder_ == null) { + region_ = builderForValue.build(); + onChanged(); + } else { + regionBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder mergeRegion(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier value) { + if (regionBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + region_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance()) { + region_ = + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.newBuilder(region_).mergeFrom(value).buildPartial(); + } else { + region_ = value; + } + onChanged(); + } else { + regionBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder clearRegion() { + if (regionBuilder_ == null) { + region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); + onChanged(); + } else { + regionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder getRegionBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getRegionFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder getRegionOrBuilder() { + if (regionBuilder_ != null) { + return regionBuilder_.getMessageOrBuilder(); + } else { + return region_; + } + } + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder> + getRegionFieldBuilder() { + if (regionBuilder_ == null) { + regionBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifierOrBuilder>( + region_, + getParentForChildren(), + isClean()); + region_ = null; + } + return regionBuilder_; + } + + // required .NameBytesPair value = 2; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> valueBuilder_; + public boolean hasValue() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getValue() { + if (valueBuilder_ == null) { + return value_; + } else { + return valueBuilder_.getMessage(); + } + } + public Builder setValue(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value) { + if (valueBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + value_ = value; + onChanged(); + } else { + valueBuilder_.setMessage(value); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder setValue( + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder builderForValue) { + if (valueBuilder_ == null) { + value_ = builderForValue.build(); + onChanged(); + } else { + valueBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder mergeValue(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value) { + if (valueBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + value_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance()) { + value_ = + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.newBuilder(value_).mergeFrom(value).buildPartial(); + } else { + value_ = value; + } + onChanged(); + } else { + valueBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000002; + return this; + } + public Builder clearValue() { + if (valueBuilder_ == null) { + value_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); + onChanged(); + } else { + valueBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder getValueBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getValueFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getValueOrBuilder() { + if (valueBuilder_ != null) { + return valueBuilder_.getMessageOrBuilder(); + } else { + return value_; + } + } + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder> + getValueFieldBuilder() { + if (valueBuilder_ == null) { + valueBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder>( + value_, + getParentForChildren(), + isClean()); + value_ = null; + } + return valueBuilder_; + } + + // @@protoc_insertion_point(builder_scope:CoprocessorServiceResponse) + } + + static { + defaultInstance = new CoprocessorServiceResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:CoprocessorServiceResponse) + } + public interface MultiActionOrBuilder extends com.google.protobuf.MessageOrBuilder { @@ -20845,6 +22808,11 @@ public final class ClientProtos { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorRequest request, com.google.protobuf.RpcCallback done); + public abstract void execService( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest request, + com.google.protobuf.RpcCallback done); + public abstract void multi( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request, @@ -20911,6 +22879,14 @@ public final class ClientProtos { impl.execCoprocessor(controller, request, done); } + @java.lang.Override + public void execService( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest request, + com.google.protobuf.RpcCallback done) { + impl.execService(controller, request, done); + } + @java.lang.Override public void multi( com.google.protobuf.RpcController controller, @@ -20956,6 +22932,8 @@ public final class ClientProtos { case 6: return impl.execCoprocessor(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorRequest)request); case 7: + return impl.execService(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest)request); + case 8: return impl.multi(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest)request); default: throw new java.lang.AssertionError("Can't get here."); @@ -20986,6 +22964,8 @@ public final class ClientProtos { case 6: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorRequest.getDefaultInstance(); case 7: + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest.getDefaultInstance(); + case 8: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -21016,6 +22996,8 @@ public final class ClientProtos { case 6: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorResponse.getDefaultInstance(); case 7: + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance(); + case 8: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -21060,6 +23042,11 @@ public final class ClientProtos { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorRequest request, com.google.protobuf.RpcCallback done); + public abstract void execService( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest request, + com.google.protobuf.RpcCallback done); + public abstract void multi( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request, @@ -21123,6 +23110,11 @@ public final class ClientProtos { done)); return; case 7: + this.execService(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 8: this.multi(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); @@ -21156,6 +23148,8 @@ public final class ClientProtos { case 6: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorRequest.getDefaultInstance(); case 7: + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest.getDefaultInstance(); + case 8: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -21186,6 +23180,8 @@ public final class ClientProtos { case 6: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorResponse.getDefaultInstance(); case 7: + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance(); + case 8: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -21313,12 +23309,27 @@ public final class ClientProtos { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorResponse.getDefaultInstance())); } + public void execService( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(7), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.class, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance())); + } + public void multi( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(7), + getDescriptor().getMethods().get(8), controller, request, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance(), @@ -21370,6 +23381,11 @@ public final class ClientProtos { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorRequest request) throws com.google.protobuf.ServiceException; + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse execService( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest request) + throws com.google.protobuf.ServiceException; + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse multi( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request) @@ -21467,12 +23483,24 @@ public final class ClientProtos { } + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse execService( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(7), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance()); + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse multi( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(7), + getDescriptor().getMethods().get(8), controller, request, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse.getDefaultInstance()); @@ -21601,6 +23629,21 @@ public final class ClientProtos { private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_ExecCoprocessorResponse_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_CoprocessorServiceCall_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_CoprocessorServiceCall_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_CoprocessorServiceRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_CoprocessorServiceRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_CoprocessorServiceResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_CoprocessorServiceResponse_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor internal_static_MultiAction_descriptor; private static @@ -21696,26 +23739,35 @@ public final class ClientProtos { "orRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpecif" + "ier\022\023\n\004call\030\002 \002(\0132\005.Exec\"8\n\027ExecCoproces" + "sorResponse\022\035\n\005value\030\001 \002(\0132\016.NameBytesPa" + - "ir\"N\n\013MultiAction\022\027\n\006mutate\030\001 \001(\0132\007.Muta" + - "te\022\021\n\003get\030\002 \001(\0132\004.Get\022\023\n\004exec\030\003 \001(\0132\005.Ex" + - "ec\"P\n\014ActionResult\022\035\n\005value\030\001 \001(\0132\016.Name" + - "BytesPair\022!\n\texception\030\002 \001(\0132\016.NameBytes", - "Pair\"^\n\014MultiRequest\022 \n\006region\030\001 \002(\0132\020.R" + - "egionSpecifier\022\034\n\006action\030\002 \003(\0132\014.MultiAc" + - "tion\022\016\n\006atomic\030\003 \001(\010\".\n\rMultiResponse\022\035\n" + - "\006result\030\001 \003(\0132\r.ActionResult2\221\003\n\rClientS" + - "ervice\022 \n\003get\022\013.GetRequest\032\014.GetResponse" + - "\022)\n\006mutate\022\016.MutateRequest\032\017.MutateRespo" + - "nse\022#\n\004scan\022\014.ScanRequest\032\r.ScanResponse" + - "\022,\n\007lockRow\022\017.LockRowRequest\032\020.LockRowRe" + - "sponse\0222\n\tunlockRow\022\021.UnlockRowRequest\032\022" + - ".UnlockRowResponse\022>\n\rbulkLoadHFile\022\025.Bu", - "lkLoadHFileRequest\032\026.BulkLoadHFileRespon" + - "se\022D\n\017execCoprocessor\022\027.ExecCoprocessorR" + - "equest\032\030.ExecCoprocessorResponse\022&\n\005mult" + - "i\022\r.MultiRequest\032\016.MultiResponseBB\n*org." + - "apache.hadoop.hbase.protobuf.generatedB\014" + - "ClientProtosH\001\210\001\001\240\001\001" + "ir\"_\n\026CoprocessorServiceCall\022\013\n\003row\030\001 \002(" + + "\014\022\023\n\013serviceName\030\002 \002(\t\022\022\n\nmethodName\030\003 \002" + + "(\t\022\017\n\007request\030\004 \002(\014\"d\n\031CoprocessorServic" + + "eRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpecifi", + "er\022%\n\004call\030\002 \002(\0132\027.CoprocessorServiceCal" + + "l\"]\n\032CoprocessorServiceResponse\022 \n\006regio" + + "n\030\001 \002(\0132\020.RegionSpecifier\022\035\n\005value\030\002 \002(\013" + + "2\016.NameBytesPair\"N\n\013MultiAction\022\027\n\006mutat" + + "e\030\001 \001(\0132\007.Mutate\022\021\n\003get\030\002 \001(\0132\004.Get\022\023\n\004e" + + "xec\030\003 \001(\0132\005.Exec\"P\n\014ActionResult\022\035\n\005valu" + + "e\030\001 \001(\0132\016.NameBytesPair\022!\n\texception\030\002 \001" + + "(\0132\016.NameBytesPair\"^\n\014MultiRequest\022 \n\006re" + + "gion\030\001 \002(\0132\020.RegionSpecifier\022\034\n\006action\030\002" + + " \003(\0132\014.MultiAction\022\016\n\006atomic\030\003 \001(\010\".\n\rMu", + "ltiResponse\022\035\n\006result\030\001 \003(\0132\r.ActionResu" + + "lt2\331\003\n\rClientService\022 \n\003get\022\013.GetRequest" + + "\032\014.GetResponse\022)\n\006mutate\022\016.MutateRequest" + + "\032\017.MutateResponse\022#\n\004scan\022\014.ScanRequest\032" + + "\r.ScanResponse\022,\n\007lockRow\022\017.LockRowReque" + + "st\032\020.LockRowResponse\0222\n\tunlockRow\022\021.Unlo" + + "ckRowRequest\032\022.UnlockRowResponse\022>\n\rbulk" + + "LoadHFile\022\025.BulkLoadHFileRequest\032\026.BulkL" + + "oadHFileResponse\022D\n\017execCoprocessor\022\027.Ex" + + "ecCoprocessorRequest\032\030.ExecCoprocessorRe", + "sponse\022F\n\013execService\022\032.CoprocessorServi" + + "ceRequest\032\033.CoprocessorServiceResponse\022&" + + "\n\005multi\022\r.MultiRequest\032\016.MultiResponseBB" + + "\n*org.apache.hadoop.hbase.protobuf.gener" + + "atedB\014ClientProtosH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -21914,8 +23966,32 @@ public final class ClientProtos { new java.lang.String[] { "Value", }, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorResponse.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorResponse.Builder.class); - internal_static_MultiAction_descriptor = + internal_static_CoprocessorServiceCall_descriptor = getDescriptor().getMessageTypes().get(21); + internal_static_CoprocessorServiceCall_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_CoprocessorServiceCall_descriptor, + new java.lang.String[] { "Row", "ServiceName", "MethodName", "Request", }, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.class, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.Builder.class); + internal_static_CoprocessorServiceRequest_descriptor = + getDescriptor().getMessageTypes().get(22); + internal_static_CoprocessorServiceRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_CoprocessorServiceRequest_descriptor, + new java.lang.String[] { "Region", "Call", }, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest.class, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest.Builder.class); + internal_static_CoprocessorServiceResponse_descriptor = + getDescriptor().getMessageTypes().get(23); + internal_static_CoprocessorServiceResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_CoprocessorServiceResponse_descriptor, + new java.lang.String[] { "Region", "Value", }, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.class, + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.Builder.class); + internal_static_MultiAction_descriptor = + getDescriptor().getMessageTypes().get(24); internal_static_MultiAction_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_MultiAction_descriptor, @@ -21923,7 +23999,7 @@ public final class ClientProtos { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiAction.Builder.class); internal_static_ActionResult_descriptor = - getDescriptor().getMessageTypes().get(22); + getDescriptor().getMessageTypes().get(25); internal_static_ActionResult_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ActionResult_descriptor, @@ -21931,7 +24007,7 @@ public final class ClientProtos { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ActionResult.Builder.class); internal_static_MultiRequest_descriptor = - getDescriptor().getMessageTypes().get(23); + getDescriptor().getMessageTypes().get(26); internal_static_MultiRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_MultiRequest_descriptor, @@ -21939,7 +24015,7 @@ public final class ClientProtos { org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest.Builder.class); internal_static_MultiResponse_descriptor = - getDescriptor().getMessageTypes().get(24); + getDescriptor().getMessageTypes().get(27); internal_static_MultiResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_MultiResponse_descriptor, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 45d1ec48330..4edff2af7a7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -61,6 +61,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; +import com.google.protobuf.*; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; @@ -115,6 +116,7 @@ import org.apache.hadoop.hbase.ipc.HBaseServer; import org.apache.hadoop.hbase.ipc.RpcCallContext; import org.apache.hadoop.hbase.monitoring.MonitoredTask; import org.apache.hadoop.hbase.monitoring.TaskMonitor; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest; import org.apache.hadoop.hbase.regionserver.metrics.OperationMetrics; import org.apache.hadoop.hbase.regionserver.metrics.RegionMetricsStorage; @@ -143,6 +145,8 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.MutableClassToInstanceMap; +import static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall; + /** * HRegion stores data for a certain region of a table. It stores all columns * for each row. A given table consists of one or more HRegions. @@ -215,6 +219,9 @@ public class HRegion implements HeapSize { // , Writable{ private Map> protocolHandlerNames = Maps.newHashMap(); + // TODO: account for each registered handler in HeapSize computation + private Map coprocessorServiceHandlers = Maps.newHashMap(); + /** * Temporary subdirectory of the region directory used for compaction output. */ @@ -5027,7 +5034,7 @@ public class HRegion implements HeapSize { // , Writable{ public static final long FIXED_OVERHEAD = ClassSize.align( ClassSize.OBJECT + ClassSize.ARRAY + - 36 * ClassSize.REFERENCE + Bytes.SIZEOF_INT + + 37 * ClassSize.REFERENCE + Bytes.SIZEOF_INT + (7 * Bytes.SIZEOF_LONG) + Bytes.SIZEOF_BOOLEAN); @@ -5085,6 +5092,7 @@ public class HRegion implements HeapSize { // , Writable{ * @return {@code true} if the registration was successful, {@code false} * otherwise */ + @Deprecated public boolean registerProtocol( Class protocol, T handler) { @@ -5108,6 +5116,41 @@ public class HRegion implements HeapSize { // , Writable{ return true; } + /** + * Registers a new protocol buffer {@link Service} subclass as a coprocessor endpoint to + * be available for handling + * {@link HRegion#execService(com.google.protobuf.RpcController, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall)}} calls. + * + *

+ * Only a single instance may be registered per region for a given {@link Service} subclass (the + * instances are keyed on {@link com.google.protobuf.Descriptors.ServiceDescriptor#getFullName()}. + * After the first registration, subsequent calls with the same service name will fail with + * a return value of {@code false}. + *

+ * @param instance the {@code Service} subclass instance to expose as a coprocessor endpoint + * @return {@code true} if the registration was successful, {@code false} + * otherwise + */ + public boolean registerService(Service instance) { + /* + * No stacking of instances is allowed for a single service name + */ + Descriptors.ServiceDescriptor serviceDesc = instance.getDescriptorForType(); + if (coprocessorServiceHandlers.containsKey(serviceDesc.getFullName())) { + LOG.error("Coprocessor service "+serviceDesc.getFullName()+ + " already registered, rejecting request from "+instance + ); + return false; + } + + coprocessorServiceHandlers.put(serviceDesc.getFullName(), instance); + if (LOG.isDebugEnabled()) { + LOG.debug("Registered coprocessor service: region="+ + Bytes.toStringBinary(getRegionName())+" service="+serviceDesc.getFullName()); + } + return true; + } + /** * Executes a single {@link org.apache.hadoop.hbase.ipc.CoprocessorProtocol} * method using the registered protocol handlers. @@ -5123,6 +5166,7 @@ public class HRegion implements HeapSize { // , Writable{ * occurs during the invocation * @see org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol) */ + @Deprecated public ExecResult exec(Exec call) throws IOException { Class protocol = call.getProtocol(); @@ -5174,6 +5218,55 @@ public class HRegion implements HeapSize { // , Writable{ return new ExecResult(getRegionName(), value); } + /** + * Executes a single protocol buffer coprocessor endpoint {@link Service} method using + * the registered protocol handlers. {@link Service} implementations must be registered via the + * {@link org.apache.hadoop.hbase.regionserver.HRegion#registerService(com.google.protobuf.Service)} + * method before they are available. + * + * @param controller an {@code RpcContoller} implementation to pass to the invoked service + * @param call a {@code CoprocessorServiceCall} instance identifying the service, method, + * and parameters for the method invocation + * @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 org.apache.hadoop.hbase.regionserver.HRegion#registerService(com.google.protobuf.Service) + */ + public Message execService(RpcController controller, CoprocessorServiceCall call) + throws IOException { + String serviceName = call.getServiceName(); + String methodName = call.getMethodName(); + if (!coprocessorServiceHandlers.containsKey(serviceName)) { + throw new HBaseRPC.UnknownProtocolException(null, + "No registered coprocessor service found for name "+serviceName+ + " in region "+Bytes.toStringBinary(getRegionName())); + } + + Service service = coprocessorServiceHandlers.get(serviceName); + Descriptors.ServiceDescriptor serviceDesc = service.getDescriptorForType(); + Descriptors.MethodDescriptor methodDesc = serviceDesc.findMethodByName(methodName); + if (methodDesc == null) { + throw new HBaseRPC.UnknownProtocolException(service.getClass(), + "Unknown method "+methodName+" called on service "+serviceName+ + " in region "+Bytes.toStringBinary(getRegionName())); + } + + Message request = service.getRequestPrototype(methodDesc).newBuilderForType() + .mergeFrom(call.getRequest()).build(); + final Message.Builder responseBuilder = + service.getResponsePrototype(methodDesc).newBuilderForType(); + service.callMethod(methodDesc, controller, request, new RpcCallback() { + @Override + public void run(Message message) { + if (message != null) { + responseBuilder.mergeFrom(message); + } + } + }); + + return responseBuilder.build(); + } + /* * Process table. * Do major compaction or list content. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 22dc66103b8..d45387b3553 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -54,6 +54,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import javax.management.ObjectName; +import com.google.protobuf.Message; import org.apache.commons.lang.mutable.MutableDouble; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -115,6 +116,7 @@ import org.apache.hadoop.hbase.ipc.HBaseRpcMetrics; import org.apache.hadoop.hbase.ipc.ProtocolSignature; import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; +import org.apache.hadoop.hbase.ipc.ServerRpcController; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.ResponseConverter; @@ -230,6 +232,9 @@ import com.google.protobuf.ByteString; import com.google.protobuf.Message; import com.google.protobuf.RpcController; +import static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest; +import static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse; + /** * HRegionServer makes a set of HRegions available to clients. It checks in with * the HMaster. There are many HRegionServers in a single HBase deployment. @@ -3333,6 +3338,31 @@ public class HRegionServer implements ClientProtocol, } } + @Override + public CoprocessorServiceResponse execService(final RpcController controller, + final CoprocessorServiceRequest request) throws ServiceException { + try { + requestCount.incrementAndGet(); + HRegion region = getRegion(request.getRegion()); + // ignore the passed in controller (from the serialized call) + ServerRpcController execController = new ServerRpcController(); + Message result = region.execService(execController, request.getCall()); + if (execController.getFailedOn() != null) { + throw execController.getFailedOn(); + } + CoprocessorServiceResponse.Builder builder = + CoprocessorServiceResponse.newBuilder(); + builder.setRegion(RequestConverter.buildRegionSpecifier( + RegionSpecifierType.REGION_NAME, region.getRegionName())); + builder.setValue( + builder.getValueBuilder().setName(result.getClass().getName()) + .setValue(result.toByteString())); + return builder.build(); + } catch (IOException ie) { + throw new ServiceException(ie); + } + } + /** * Execute multiple actions on a table: get, mutate, and/or execCoprocessor * diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java index 50bc1a5361f..ed14364cbec 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionCoprocessorHost.java @@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.coprocessor.CoprocessorService; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionObserver; @@ -216,7 +217,11 @@ public class RegionCoprocessorHost for (Class c : implClass.getInterfaces()) { if (CoprocessorProtocol.class.isAssignableFrom(c)) { region.registerProtocol(c, (CoprocessorProtocol)instance); - break; + } + // we allow endpoints to register as both CoproocessorProtocols and Services + // for ease of transition + if (CoprocessorService.class.isAssignableFrom(c)) { + region.registerService( ((CoprocessorService)instance).getService() ); } } ConcurrentMap classData; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java index ae661147fc1..d8e0df6ffd1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java @@ -28,12 +28,15 @@ import java.util.Map; import java.util.Set; import java.util.TreeMap; +import com.google.protobuf.Service; +import com.google.protobuf.ServiceException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.client.*; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.ipc.CoprocessorProtocol; +import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; @@ -732,6 +735,25 @@ public class RemoteHTable implements HTableInterface { throw new UnsupportedOperationException("coprocessorExec not implemented"); } + @Override + public CoprocessorRpcChannel coprocessorService(byte[] row) { + throw new UnsupportedOperationException("coprocessorService not implemented"); + } + + @Override + public Map coprocessorService(Class service, + byte[] startKey, byte[] endKey, Batch.Call callable) + throws ServiceException, Throwable { + throw new UnsupportedOperationException("coprocessorService not implemented"); + } + + @Override + public void coprocessorService(Class service, + byte[] startKey, byte[] endKey, Batch.Call callable, Batch.Callback callback) + throws ServiceException, Throwable { + throw new UnsupportedOperationException("coprocessorService not implemented"); + } + @Override public void mutateRow(RowMutations rm) throws IOException { throw new IOException("atomicMutation not supported"); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java index 08b811d8a0b..2dd977149d1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java @@ -24,6 +24,9 @@ import java.util.Map; import java.util.Set; import java.util.TreeSet; +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -40,18 +43,16 @@ import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; -import org.apache.hadoop.hbase.coprocessor.CoprocessorException; -import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; -import org.apache.hadoop.hbase.coprocessor.MasterObserver; -import org.apache.hadoop.hbase.coprocessor.ObserverContext; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.*; import org.apache.hadoop.hbase.filter.CompareFilter; import org.apache.hadoop.hbase.filter.FilterList; import org.apache.hadoop.hbase.filter.ByteArrayComparable; import org.apache.hadoop.hbase.ipc.HBaseRPC; import org.apache.hadoop.hbase.ipc.ProtocolSignature; import org.apache.hadoop.hbase.ipc.RequestContext; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.ResponseConverter; +import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.InternalScanner; import org.apache.hadoop.hbase.regionserver.RegionScanner; @@ -69,6 +70,8 @@ import com.google.common.collect.MapMaker; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService; + /** * Provides basic authorization checks for data access and administrative * operations. @@ -101,7 +104,8 @@ import com.google.common.collect.Sets; *

*/ public class AccessController extends BaseRegionObserver - implements MasterObserver, AccessControllerProtocol { + implements MasterObserver, AccessControllerProtocol, + AccessControlService.Interface, CoprocessorService { /** * Represents the result of an authorization check for logging and error * reporting. @@ -1049,6 +1053,7 @@ public class AccessController extends BaseRegionObserver * These methods are only allowed to be called against the _acl_ region(s). * This will be restricted by both client side and endpoint implementations. */ + @Deprecated @Override public void grant(UserPermission perm) throws IOException { // verify it's only running at .acl. @@ -1079,6 +1084,7 @@ public class AccessController extends BaseRegionObserver permission.getActions())); } + @Deprecated @Override public void revoke(UserPermission perm) throws IOException { // only allowed to be called on _acl_ region @@ -1109,6 +1115,7 @@ public class AccessController extends BaseRegionObserver permission.getActions())); } + @Deprecated @Override public List getUserPermissions(final byte[] tableName) throws IOException { // only allowed to be called on _acl_ region @@ -1124,6 +1131,7 @@ public class AccessController extends BaseRegionObserver } } + @Deprecated @Override public void checkPermissions(Permission[] permissions) throws IOException { byte[] tableName = regionEnv.getRegion().getTableDesc().getName(); @@ -1158,11 +1166,13 @@ public class AccessController extends BaseRegionObserver } } + @Deprecated @Override public long getProtocolVersion(String protocol, long clientVersion) throws IOException { return PROTOCOL_VERSION; } + @Deprecated @Override public ProtocolSignature getProtocolSignature(String protocol, long clientVersion, int clientMethodsHash) throws IOException { @@ -1173,6 +1183,79 @@ public class AccessController extends BaseRegionObserver "Unexpected protocol requested: "+protocol); } + + /* ---- Protobuf AccessControlService implementation ---- */ + @Override + public void grant(RpcController controller, + AccessControlProtos.GrantRequest request, + RpcCallback done) { + UserPermission perm = ProtobufUtil.toUserPermission(request.getPermission()); + AccessControlProtos.GrantResponse response = null; + try { + grant(perm); + response = AccessControlProtos.GrantResponse.getDefaultInstance(); + } catch (IOException ioe) { + // pass exception back up + ResponseConverter.setControllerException(controller, ioe); + } + done.run(response); + } + + @Override + public void revoke(RpcController controller, + AccessControlProtos.RevokeRequest request, + RpcCallback done) { + UserPermission perm = ProtobufUtil.toUserPermission(request.getPermission()); + AccessControlProtos.RevokeResponse response = null; + try { + revoke(perm); + response = AccessControlProtos.RevokeResponse.getDefaultInstance(); + } catch (IOException ioe) { + // pass exception back up + ResponseConverter.setControllerException(controller, ioe); + } + done.run(response); + } + + @Override + public void getUserPermissions(RpcController controller, + AccessControlProtos.UserPermissionsRequest request, + RpcCallback done) { + byte[] table = request.getTable().toByteArray(); + AccessControlProtos.UserPermissionsResponse response = null; + try { + List perms = getUserPermissions(table); + response = ResponseConverter.buildUserPermissionsResponse(perms); + } catch (IOException ioe) { + // pass exception back up + ResponseConverter.setControllerException(controller, ioe); + } + done.run(response); + } + + @Override + public void checkPermissions(RpcController controller, + AccessControlProtos.CheckPermissionsRequest request, + RpcCallback done) { + Permission[] perms = new Permission[request.getPermissionCount()]; + for (int i=0; i < request.getPermissionCount(); i++) { + perms[i] = ProtobufUtil.toPermission(request.getPermission(i)); + } + AccessControlProtos.CheckPermissionsResponse response = null; + try { + checkPermissions(perms); + response = AccessControlProtos.CheckPermissionsResponse.getDefaultInstance(); + } catch (IOException ioe) { + ResponseConverter.setControllerException(controller, ioe); + } + done.run(response); + } + + @Override + public Service getService() { + return AccessControlProtos.AccessControlService.newReflectiveService(this); + } + private byte[] getTableName(RegionCoprocessorEnvironment e) { HRegion region = e.getRegion(); byte[] tableName = null; diff --git a/hbase-server/src/main/protobuf/AccessControl.proto b/hbase-server/src/main/protobuf/AccessControl.proto index 7bdf1bcbccf..ea772827b52 100644 --- a/hbase-server/src/main/protobuf/AccessControl.proto +++ b/hbase-server/src/main/protobuf/AccessControl.proto @@ -52,3 +52,48 @@ message UserTablePermissions { repeated UserPermissions permissions = 1; } + +message GrantRequest { + required UserPermission permission = 1; +} + +message GrantResponse { +} + +message RevokeRequest { + required UserPermission permission = 1; + +} + +message RevokeResponse { +} + + +message UserPermissionsRequest { + required bytes table = 1; +} + +message UserPermissionsResponse { + repeated UserPermission permission = 1; +} + +message CheckPermissionsRequest { + repeated Permission permission = 1; +} + +message CheckPermissionsResponse { +} + +service AccessControlService { + rpc grant(GrantRequest) + returns (GrantResponse); + + rpc revoke(RevokeRequest) + returns (RevokeResponse); + + rpc getUserPermissions(UserPermissionsRequest) + returns (UserPermissionsResponse); + + rpc checkPermissions(CheckPermissionsRequest) + returns (CheckPermissionsResponse); +} diff --git a/hbase-server/src/main/protobuf/Client.proto b/hbase-server/src/main/protobuf/Client.proto index 0952d83035a..c05a33d6c25 100644 --- a/hbase-server/src/main/protobuf/Client.proto +++ b/hbase-server/src/main/protobuf/Client.proto @@ -296,6 +296,23 @@ message ExecCoprocessorResponse { required NameBytesPair value = 1; } +message CoprocessorServiceCall { + required bytes row = 1; + required string serviceName = 2; + required string methodName = 3; + required bytes request = 4; +} + +message CoprocessorServiceRequest { + required RegionSpecifier region = 1; + required CoprocessorServiceCall call = 2; +} + +message CoprocessorServiceResponse { + required RegionSpecifier region = 1; + required NameBytesPair value = 2; +} + /** * An action that is part of MultiRequest. * This is a union type - exactly one of the fields will be set. @@ -359,6 +376,9 @@ service ClientService { rpc execCoprocessor(ExecCoprocessorRequest) returns(ExecCoprocessorResponse); + rpc execService(CoprocessorServiceRequest) + returns(CoprocessorServiceResponse); + rpc multi(MultiRequest) returns(MultiResponse); } diff --git a/hbase-server/src/main/protobuf/Examples.proto b/hbase-server/src/main/protobuf/Examples.proto new file mode 100644 index 00000000000..24fbad6ebd7 --- /dev/null +++ b/hbase-server/src/main/protobuf/Examples.proto @@ -0,0 +1,37 @@ +/** + * 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. + */ + +option java_package = "org.apache.hadoop.hbase.coprocessor.example.generated"; +option java_outer_classname = "ExampleProtos"; +option java_generic_services = true; +option java_generate_equals_and_hash = true; +option optimize_for = SPEED; + +message CountRequest { +} + +message CountResponse { + required int64 count = 1 [default = 0]; +} + +service RowCountService { + rpc getRowCount(CountRequest) + returns (CountResponse); + rpc getKeyValueCount(CountRequest) + returns (CountResponse); +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java index 5897e5c63af..e75a43d18ea 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java @@ -46,13 +46,16 @@ implements ColumnAggregationProtocol { .getRegion().getScanner(scan); try { List curVals = new ArrayList(); - boolean done = false; + boolean hasMore = false; do { curVals.clear(); - done = scanner.next(curVals); - KeyValue kv = curVals.get(0); - sumResult += Bytes.toInt(kv.getBuffer(), kv.getValueOffset()); - } while (done); + hasMore = scanner.next(curVals); + for (KeyValue kv : curVals) { + if (Bytes.equals(qualifier, kv.getQualifier())) { + sumResult += Bytes.toInt(kv.getBuffer(), kv.getValueOffset()); + } + } + } while (hasMore); } finally { scanner.close(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ProtobufCoprocessorService.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ProtobufCoprocessorService.java new file mode 100644 index 00000000000..cdda28ab538 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ProtobufCoprocessorService.java @@ -0,0 +1,77 @@ +/* + * 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.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; +import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos; +import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos; +import org.apache.hadoop.hbase.protobuf.ResponseConverter; + +import java.io.IOException; + +/** + * Test implementation of a coprocessor endpoint exposing the + * {@link TestRpcServiceProtos.TestProtobufRpcProto} service methods. For internal use by + * unit tests only. + */ +public class ProtobufCoprocessorService + extends TestRpcServiceProtos.TestProtobufRpcProto + implements CoprocessorService, Coprocessor { + public ProtobufCoprocessorService() { + } + + @Override + public Service getService() { + return this; + } + + @Override + public void ping(RpcController controller, TestProtos.EmptyRequestProto request, + RpcCallback done) { + done.run(TestProtos.EmptyResponseProto.getDefaultInstance()); + } + + @Override + public void echo(RpcController controller, TestProtos.EchoRequestProto request, + RpcCallback done) { + String message = request.getMessage(); + done.run(TestProtos.EchoResponseProto.newBuilder().setMessage(message).build()); + } + + @Override + public void error(RpcController controller, TestProtos.EmptyRequestProto request, + RpcCallback done) { + ResponseConverter.setControllerException(controller, new IOException("Test exception")); + done.run(null); + } + + @Override + public void start(CoprocessorEnvironment env) throws IOException { + //To change body of implemented methods use File | Settings | File Templates. + } + + @Override + public void stop(CoprocessorEnvironment env) throws IOException { + //To change body of implemented methods use File | Settings | File Templates. + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java index 7fcf1329c95..9550f4f4656 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestCoprocessorEndpoint.java @@ -18,43 +18,40 @@ */ package org.apache.hadoop.hbase.coprocessor; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; - import java.io.IOException; +import java.util.Collections; import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; +import com.google.protobuf.RpcController; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseConfiguration; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.MediumTests; -import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.*; +import org.apache.hadoop.hbase.client.HBaseAdmin; import org.apache.hadoop.hbase.client.HTable; import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.coprocessor.Batch; -import org.apache.hadoop.hbase.client.coprocessor.Exec; -import org.apache.hadoop.hbase.io.HbaseObjectWritable; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; +import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; +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.util.Bytes; -import org.apache.hadoop.io.DataInputBuffer; -import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.Text; import org.junit.AfterClass; import org.junit.BeforeClass; -import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; -import com.google.protobuf.ByteString; +import static org.junit.Assert.*; /** * TestEndpoint: test cases to verify coprocessor Endpoint */ @Category(MediumTests.class) public class TestCoprocessorEndpoint { + private static final Log LOG = LogFactory.getLog(TestCoprocessorEndpoint.class); private static final byte[] TEST_TABLE = Bytes.toBytes("TestTable"); private static final byte[] TEST_FAMILY = Bytes.toBytes("TestFamily"); @@ -76,27 +73,23 @@ public class TestCoprocessorEndpoint { // set configure to indicate which cp should be loaded Configuration conf = util.getConfiguration(); conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, - "org.apache.hadoop.hbase.coprocessor.ColumnAggregationEndpoint", - "org.apache.hadoop.hbase.coprocessor.GenericEndpoint"); + org.apache.hadoop.hbase.coprocessor.ColumnAggregationEndpoint.class.getName(), + org.apache.hadoop.hbase.coprocessor.GenericEndpoint.class.getName(), + ProtobufCoprocessorService.class.getName()); util.startMiniCluster(2); - HTable table = util.createTable(TEST_TABLE, TEST_FAMILY); - util.createMultiRegions(util.getConfiguration(), table, TEST_FAMILY, - new byte[][] { HConstants.EMPTY_BYTE_ARRAY, - ROWS[rowSeperator1], ROWS[rowSeperator2] }); + HBaseAdmin admin = new HBaseAdmin(conf); + HTableDescriptor desc = new HTableDescriptor(TEST_TABLE); + desc.addFamily(new HColumnDescriptor(TEST_FAMILY)); + admin.createTable(desc, new byte[][]{ROWS[rowSeperator1], ROWS[rowSeperator2]}); + util.waitUntilAllRegionsAssigned(3); + admin.close(); + HTable table = new HTable(conf, TEST_TABLE); for (int i = 0; i < ROWSIZE; i++) { Put put = new Put(ROWS[i]); - put.setWriteToWAL(false); put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(i)); table.put(put); } - - // sleep here is an ugly hack to allow region transitions to finish - long timeout = System.currentTimeMillis() + (15 * 1000); - while ((System.currentTimeMillis() < timeout) && - (table.getRegionsInfo().size() != 2)) { - Thread.sleep(250); - } table.close(); } @@ -135,7 +128,7 @@ public class TestCoprocessorEndpoint { table.close(); } - @Ignore @Test + @Test public void testAggregation() throws Throwable { HTable table = new HTable(util.getConfiguration(), TEST_TABLE); Map results; @@ -143,7 +136,7 @@ public class TestCoprocessorEndpoint { // scan: for all regions results = table .coprocessorExec(ColumnAggregationProtocol.class, - ROWS[rowSeperator1 - 1], ROWS[rowSeperator2 + 1], + ROWS[0], ROWS[ROWS.length-1], new Batch.Call() { public Long call(ColumnAggregationProtocol instance) throws IOException { @@ -153,19 +146,20 @@ public class TestCoprocessorEndpoint { int sumResult = 0; int expectedResult = 0; for (Map.Entry e : results.entrySet()) { + LOG.info("Got value "+e.getValue()+" for region "+Bytes.toStringBinary(e.getKey())); sumResult += e.getValue(); } for (int i = 0; i < ROWSIZE; i++) { expectedResult += i; } - assertEquals("Invalid result", sumResult, expectedResult); + assertEquals("Invalid result", expectedResult, sumResult); results.clear(); // scan: for region 2 and region 3 results = table .coprocessorExec(ColumnAggregationProtocol.class, - ROWS[rowSeperator1 + 1], ROWS[rowSeperator2 + 1], + ROWS[rowSeperator1], ROWS[ROWS.length-1], new Batch.Call() { public Long call(ColumnAggregationProtocol instance) throws IOException { @@ -175,15 +169,90 @@ public class TestCoprocessorEndpoint { sumResult = 0; expectedResult = 0; for (Map.Entry e : results.entrySet()) { + LOG.info("Got value "+e.getValue()+" for region "+Bytes.toStringBinary(e.getKey())); sumResult += e.getValue(); } for (int i = rowSeperator1; i < ROWSIZE; i++) { expectedResult += i; } - assertEquals("Invalid result", sumResult, expectedResult); + assertEquals("Invalid result", expectedResult, sumResult); table.close(); } + @Test + public void testCoprocessorService() throws Throwable { + HTable table = new HTable(util.getConfiguration(), TEST_TABLE); + NavigableMap regions = table.getRegionLocations(); + + final TestProtos.EchoRequestProto request = + TestProtos.EchoRequestProto.newBuilder().setMessage("hello").build(); + final Map results = Collections.synchronizedMap( + new TreeMap(Bytes.BYTES_COMPARATOR)); + try { + // scan: for all regions + final RpcController controller = new ServerRpcController(); + table.coprocessorService(TestRpcServiceProtos.TestProtobufRpcProto.class, + ROWS[0], ROWS[ROWS.length - 1], + new Batch.Call() { + public TestProtos.EchoResponseProto call(TestRpcServiceProtos.TestProtobufRpcProto instance) + throws IOException { + LOG.debug("Default response is " + TestProtos.EchoRequestProto.getDefaultInstance()); + BlockingRpcCallback callback = new BlockingRpcCallback(); + instance.echo(controller, request, callback); + TestProtos.EchoResponseProto response = callback.get(); + LOG.debug("Batch.Call returning result " + response); + return response; + } + }, + new Batch.Callback() { + public void update(byte[] region, byte[] row, TestProtos.EchoResponseProto result) { + assertNotNull(result); + assertEquals("hello", result.getMessage()); + results.put(region, result.getMessage()); + } + } + ); + for (Map.Entry e : results.entrySet()) { + LOG.info("Got value "+e.getValue()+" for region "+Bytes.toStringBinary(e.getKey())); + } + assertEquals(3, results.size()); + for (HRegionInfo info : regions.navigableKeySet()) { + LOG.info("Region info is "+info.getRegionNameAsString()); + assertTrue(results.containsKey(info.getRegionName())); + } + results.clear(); + + // scan: for region 2 and region 3 + table.coprocessorService(TestRpcServiceProtos.TestProtobufRpcProto.class, + ROWS[rowSeperator1], ROWS[ROWS.length - 1], + new Batch.Call() { + public TestProtos.EchoResponseProto call(TestRpcServiceProtos.TestProtobufRpcProto instance) + throws IOException { + LOG.debug("Default response is " + TestProtos.EchoRequestProto.getDefaultInstance()); + BlockingRpcCallback callback = new BlockingRpcCallback(); + instance.echo(controller, request, callback); + TestProtos.EchoResponseProto response = callback.get(); + LOG.debug("Batch.Call returning result " + response); + return response; + } + }, + new Batch.Callback() { + public void update(byte[] region, byte[] row, TestProtos.EchoResponseProto result) { + assertNotNull(result); + assertEquals("hello", result.getMessage()); + results.put(region, result.getMessage()); + } + } + ); + for (Map.Entry e : results.entrySet()) { + LOG.info("Got value "+e.getValue()+" for region "+Bytes.toStringBinary(e.getKey())); + } + assertEquals(2, results.size()); + } finally { + table.close(); + } + } + private static byte[][] makeN(byte[] base, int n) { byte[][] ret = new byte[n][]; for (int i = 0; i < n; i++) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java new file mode 100644 index 00000000000..eed1074f466 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java @@ -0,0 +1,111 @@ +/* + * 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.example; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.MediumTests; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.coprocessor.Batch; +import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos; +import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; +import org.apache.hadoop.hbase.ipc.ServerRpcController; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; + +import static junit.framework.Assert.*; + +/** + * Test case demonstrating client interactions with the {@link RowCountEndpoint} + * sample coprocessor Service implementation. + */ +@Category(MediumTests.class) +public class TestRowCountEndpoint { + private static final byte[] TEST_TABLE = Bytes.toBytes("testrowcounter"); + private static final byte[] TEST_FAMILY = Bytes.toBytes("f"); + private static final byte[] TEST_COLUMN = Bytes.toBytes("col"); + + private static HBaseTestingUtility TEST_UTIL = null; + private static Configuration CONF = null; + + @BeforeClass + public static void setupBeforeClass() throws Exception { + TEST_UTIL = new HBaseTestingUtility(); + CONF = TEST_UTIL.getConfiguration(); + CONF.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, + RowCountEndpoint.class.getName()); + + TEST_UTIL.startMiniCluster(); + TEST_UTIL.createTable(TEST_TABLE, TEST_FAMILY); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testEndpoint() throws Throwable { + HTable table = new HTable(CONF, TEST_TABLE); + + // insert some test rows + for (int i=0; i<5; i++) { + byte[] iBytes = Bytes.toBytes(i); + Put p = new Put(iBytes); + p.add(TEST_FAMILY, TEST_COLUMN, iBytes); + table.put(p); + } + + final ExampleProtos.CountRequest request = ExampleProtos.CountRequest.getDefaultInstance(); + Map results = table.coprocessorService(ExampleProtos.RowCountService.class, + null, null, + new Batch.Call() { + public Long call(ExampleProtos.RowCountService counter) throws IOException { + ServerRpcController controller = new ServerRpcController(); + BlockingRpcCallback rpcCallback = + new BlockingRpcCallback(); + counter.getRowCount(controller, request, rpcCallback); + ExampleProtos.CountResponse response = rpcCallback.get(); + if (controller.failedOnException()) { + throw controller.getFailedOn(); + } + return (response != null && response.hasCount()) ? response.getCount() : 0; + } + }); + // should be one region with results + assertEquals(1, results.size()); + Iterator iter = results.values().iterator(); + Long val = iter.next(); + assertNotNull(val); + assertEquals(5l, val.longValue()); + } + + @org.junit.Rule + public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu = + new org.apache.hadoop.hbase.ResourceCheckerJUnitRule(); +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java index 247437fb9c7..7def88892df 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java @@ -62,6 +62,7 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionRequest import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.SplitRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerResponse; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileResponse; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ExecCoprocessorRequest; @@ -412,6 +413,12 @@ class MockRegionServer implements AdminProtocol, ClientProtocol, RegionServerSer return null; } + @Override + public ClientProtos.CoprocessorServiceResponse execService(RpcController controller, + ClientProtos.CoprocessorServiceRequest request) throws ServiceException { + return null; + } + @Override public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiResponse multi( RpcController controller, MultiRequest request) throws ServiceException { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java index 7c70f9e0d95..80e3c326056 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java @@ -18,6 +18,8 @@ package org.apache.hadoop.hbase.security.access; +import static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService; +import static org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -27,6 +29,11 @@ import java.security.PrivilegedExceptionAction; import java.util.List; import java.util.Map; +import com.google.common.collect.Lists; +import com.google.protobuf.BlockingRpcChannel; +import com.google.protobuf.ByteString; +import com.google.protobuf.RpcChannel; +import com.google.protobuf.ServiceException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.HBaseTestingUtility; @@ -52,6 +59,8 @@ import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.master.MasterCoprocessorHost; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost; import org.apache.hadoop.hbase.security.AccessDeniedException; @@ -128,26 +137,32 @@ public class TestAccessController { // initilize access control HTable meta = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); - AccessControllerProtocol protocol = meta.coprocessorProxy(AccessControllerProtocol.class, - TEST_TABLE); + BlockingRpcChannel service = meta.coprocessorService(TEST_TABLE); + AccessControlService.BlockingInterface protocol = + AccessControlService.newBlockingStub(service); HRegion region = TEST_UTIL.getHBaseCluster().getRegions(TEST_TABLE).get(0); RegionCoprocessorHost rcpHost = region.getCoprocessorHost(); RCP_ENV = rcpHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER, Coprocessor.PRIORITY_HIGHEST, 1, conf); - protocol.grant(new UserPermission(Bytes.toBytes(USER_ADMIN.getShortName()), - Permission.Action.ADMIN, Permission.Action.CREATE, Permission.Action.READ, - Permission.Action.WRITE)); + protocol.grant(null, newGrantRequest(USER_ADMIN.getShortName(), + null, null, null, + AccessControlProtos.Permission.Action.ADMIN, + AccessControlProtos.Permission.Action.CREATE, + AccessControlProtos.Permission.Action.READ, + AccessControlProtos.Permission.Action.WRITE)); - protocol.grant(new UserPermission(Bytes.toBytes(USER_RW.getShortName()), TEST_TABLE, - TEST_FAMILY, Permission.Action.READ, Permission.Action.WRITE)); + protocol.grant(null, newGrantRequest(USER_RW.getShortName(), + TEST_TABLE, TEST_FAMILY, null, + AccessControlProtos.Permission.Action.READ, + AccessControlProtos.Permission.Action.WRITE)); - protocol.grant(new UserPermission(Bytes.toBytes(USER_RO.getShortName()), TEST_TABLE, - TEST_FAMILY, Permission.Action.READ)); + protocol.grant(null, newGrantRequest(USER_RO.getShortName(), TEST_TABLE, + TEST_FAMILY, null, AccessControlProtos.Permission.Action.READ)); - protocol.grant(new UserPermission(Bytes.toBytes(USER_CREATE.getShortName()), TEST_TABLE, null, - Permission.Action.CREATE)); + protocol.grant(null, newGrantRequest(USER_CREATE.getShortName(), + TEST_TABLE, null, null, AccessControlProtos.Permission.Action.CREATE)); } @AfterClass @@ -155,6 +170,32 @@ public class TestAccessController { TEST_UTIL.shutdownMiniCluster(); } + private static AccessControlProtos.GrantRequest newGrantRequest( + String username, byte[] table, byte[] family, byte[] qualifier, + AccessControlProtos.Permission.Action... actions) { + AccessControlProtos.Permission.Builder permissionBuilder = + AccessControlProtos.Permission.newBuilder(); + for (AccessControlProtos.Permission.Action a : actions) { + permissionBuilder.addAction(a); + } + if (table != null) { + permissionBuilder.setTable(ByteString.copyFrom(table)); + } + if (family != null) { + permissionBuilder.setFamily(ByteString.copyFrom(family)); + } + if (qualifier != null) { + permissionBuilder.setQualifier(ByteString.copyFrom(qualifier)); + } + + return AccessControlProtos.GrantRequest.newBuilder() + .setPermission( + AccessControlProtos.UserPermission.newBuilder() + .setUser(ByteString.copyFromUtf8(username)) + .setPermission(permissionBuilder.build()) + ).build(); + } + public void verifyAllowed(User user, PrivilegedExceptionAction... actions) throws Exception { for (PrivilegedExceptionAction action : actions) { try { @@ -182,7 +223,13 @@ public class TestAccessController { // AccessDeniedException boolean isAccessDeniedException = false; for (Throwable ex : e.getCauses()) { - if (ex instanceof AccessDeniedException) { + if (ex instanceof ServiceException) { + ServiceException se = (ServiceException)ex; + if (se.getCause() != null && se.getCause() instanceof AccessDeniedException) { + isAccessDeniedException = true; + break; + } + } else if (ex instanceof AccessDeniedException) { isAccessDeniedException = true; break; } @@ -1117,15 +1164,25 @@ public class TestAccessController { public void checkGlobalPerms(Permission.Action... actions) throws IOException { HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); - AccessControllerProtocol protocol = acl.coprocessorProxy(AccessControllerProtocol.class, - new byte[0]); + BlockingRpcChannel channel = acl.coprocessorService(new byte[0]); + AccessControlService.BlockingInterface protocol = + AccessControlService.newBlockingStub(channel); Permission[] perms = new Permission[actions.length]; for (int i = 0; i < actions.length; i++) { perms[i] = new Permission(actions[i]); } - protocol.checkPermissions(perms); + CheckPermissionsRequest.Builder request = CheckPermissionsRequest.newBuilder(); + for (Action a : actions) { + request.addPermission(AccessControlProtos.Permission.newBuilder() + .addAction(ProtobufUtil.toPermissionAction(a)).build()); + } + try { + protocol.checkPermissions(null, request.build()); + } catch (ServiceException se) { + ProtobufUtil.toIOException(se); + } } public void checkTablePerms(byte[] table, byte[] family, byte[] column, @@ -1140,22 +1197,39 @@ public class TestAccessController { public void checkTablePerms(byte[] table, Permission... perms) throws IOException { HTable acl = new HTable(conf, table); - AccessControllerProtocol protocol = acl.coprocessorProxy(AccessControllerProtocol.class, - new byte[0]); - - protocol.checkPermissions(perms); + AccessControlService.BlockingInterface protocol = + AccessControlService.newBlockingStub(acl.coprocessorService(new byte[0])); + CheckPermissionsRequest.Builder request = CheckPermissionsRequest.newBuilder(); + for (Permission p : perms) { + request.addPermission(ProtobufUtil.toPermission(p)); + } + try { + protocol.checkPermissions(null, request.build()); + } catch (ServiceException se) { + ProtobufUtil.toIOException(se); + } } - public void grant(AccessControllerProtocol protocol, User user, byte[] t, byte[] f, byte[] q, - Permission.Action... actions) throws IOException { - protocol.grant(new UserPermission(Bytes.toBytes(user.getShortName()), t, f, q, actions)); + public void grant(AccessControlService.BlockingInterface protocol, User user, + byte[] t, byte[] f, byte[] q, Permission.Action... actions) + throws ServiceException { + List permActions = + Lists.newArrayListWithCapacity(actions.length); + for (Action a : actions) { + permActions.add(ProtobufUtil.toPermissionAction(a)); + } + AccessControlProtos.GrantRequest request = + newGrantRequest(user.getShortName(), t, f, q, permActions.toArray( + new AccessControlProtos.Permission.Action[actions.length])); + protocol.grant(null, request); } @Test public void testCheckPermissions() throws Exception { final HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME); - final AccessControllerProtocol protocol = acl.coprocessorProxy(AccessControllerProtocol.class, - TEST_TABLE); + BlockingRpcChannel channel = acl.coprocessorService(new byte[0]); + AccessControlService.BlockingInterface protocol = + AccessControlService.newBlockingStub(channel); // -------------------------------------- // test global permissions @@ -1278,11 +1352,15 @@ public class TestAccessController { // -------------------------------------- // check for wrong table region try { + CheckPermissionsRequest checkRequest = + CheckPermissionsRequest.newBuilder().addPermission( + AccessControlProtos.Permission.newBuilder() + .setTable(ByteString.copyFrom(TEST_TABLE)).addAction(AccessControlProtos.Permission.Action.CREATE) + ).build(); // but ask for TablePermissions for TEST_TABLE - protocol.checkPermissions(new Permission[] { (Permission) new TablePermission(TEST_TABLE, - null, (byte[]) null, Permission.Action.CREATE) }); + protocol.checkPermissions(null, checkRequest); fail("this should have thrown CoprocessorException"); - } catch (CoprocessorException ex) { + } catch (ServiceException ex) { // expected }