diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index b72e982..438facb 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -36,12 +36,14 @@ import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import com.google.protobuf.InvalidProtocolBufferException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; @@ -52,6 +54,7 @@ import org.apache.hadoop.hbase.KeyValueUtil; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.coprocessor.Batch; +import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback; import org.apache.hadoop.hbase.filter.BinaryComparator; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; @@ -59,6 +62,7 @@ import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; import org.apache.hadoop.hbase.ipc.RegionCoprocessorRpcChannel; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.RequestConverter; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse; @@ -68,6 +72,8 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Threads; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; import com.google.protobuf.Service; import com.google.protobuf.ServiceException; @@ -1427,8 +1433,7 @@ public class HTable implements HTableInterface { public static void setRegionCachePrefetch( final TableName tableName, final boolean enable) throws IOException { - HConnectionManager.execute(new HConnectable(HBaseConfiguration - .create()) { + HConnectionManager.execute(new HConnectable(HBaseConfiguration.create()) { @Override public Void connect(HConnection connection) throws IOException { connection.setRegionCachePrefetch(tableName, enable); @@ -1625,4 +1630,107 @@ public class HTable implements HTableInterface { t.close(); } } + + /** + * {@inheritDoc} + */ + @Override + public Map batchCoprocessorService( + Descriptors.MethodDescriptor methodDescriptor, Message request, + byte[] startKey, byte[] endKey, R responsePrototype) throws ServiceException, Throwable { + final Map results = Collections.synchronizedMap(new TreeMap( + Bytes.BYTES_COMPARATOR)); + batchCoprocessorService(methodDescriptor, request, startKey, endKey, responsePrototype, + new Callback() { + + @Override + public void update(byte[] region, byte[] row, R result) { + if (region != null) { + results.put(region, result); + } + } + }); + return results; + } + + /** + * {@inheritDoc} + */ + @Override + public void batchCoprocessorService( + final Descriptors.MethodDescriptor methodDescriptor, final Message request, + byte[] startKey, byte[] endKey, final R responsePrototype, final Callback callback) + throws ServiceException, Throwable { + + // get regions covered by the row range + Pair, List> keysAndRegions = + getKeysAndRegionsInRange(startKey, endKey, true); + List keys = keysAndRegions.getFirst(); + List regions = keysAndRegions.getSecond(); + + List execs = new ArrayList(); + for (int i = 0; i < keys.size(); i++) { + final byte[] rowKey = keys.get(i); + final byte[] region = regions.get(i).getRegionInfo().getRegionName(); + execs.add(new RegionCoprocessorServiceExec(region, rowKey, methodDescriptor, request)); + } + + // tracking for any possible deserialization errors on success callback + // TODO: it would be better to be able to reuse AsyncProcess.BatchErrors here + final List callbackErrorExceptions = new ArrayList(); + final List callbackErrorActions = new ArrayList(); + final List callbackErrorServers = new ArrayList(); + + AsyncProcess asyncProcess = + new AsyncProcess(connection, tableName, pool, + new AsyncProcess.AsyncProcessCallback() { + @Override + public void success(int originalIndex, byte[] region, Row row, + ClientProtos.CoprocessorServiceResult serviceResult) { + if (LOG.isTraceEnabled()) { + LOG.trace("Received result for endpoint " + methodDescriptor.getFullName() + + " call #" + originalIndex + ": region=" + Bytes.toStringBinary(region) + + ", row=" + Bytes.toStringBinary(row.getRow()) + + ", value=" + serviceResult.getValue().getValue()); + } + try { + callback.update(region, row.getRow(), + (R) responsePrototype.newBuilderForType().mergeFrom( + serviceResult.getValue().getValue()).build()); + } catch (InvalidProtocolBufferException e) { + LOG.error("Unexpected response type from endpoint " + methodDescriptor.getFullName(), + e); + callbackErrorExceptions.add(e); + callbackErrorActions.add(row); + callbackErrorServers.add("null"); + } + } + + @Override + public boolean failure(int originalIndex, Row row, Throwable t) { + RegionCoprocessorServiceExec exec = (RegionCoprocessorServiceExec) row; + LOG.error("Failed calling endpoint " + methodDescriptor.getFullName() + ": region=" + + Bytes.toStringBinary(exec.getRegion()), t); + return true; + } + + @Override + public boolean retriableFailure(int originalIndex, Row row, Throwable exception) { + RegionCoprocessorServiceExec exec = (RegionCoprocessorServiceExec) row; + LOG.error("Failed calling endpoint " + methodDescriptor.getFullName() + ": region=" + + Bytes.toStringBinary(exec.getRegion()), exception); + return !(exception instanceof DoNotRetryIOException); + } + }, configuration, RpcRetryingCallerFactory.instantiate(configuration)); + + asyncProcess.submitAll(execs); + asyncProcess.waitUntilDone(); + + if (asyncProcess.hasError()) { + throw asyncProcess.getErrors(); + } else if (!callbackErrorExceptions.isEmpty()) { + throw new RetriesExhaustedWithDetailsException(callbackErrorExceptions, callbackErrorActions, + callbackErrorServers); + } + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java index d7126f0..a9a9982 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java @@ -18,6 +18,8 @@ */ package org.apache.hadoop.hbase.client; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; import com.google.protobuf.Service; import com.google.protobuf.ServiceException; @@ -591,4 +593,69 @@ public interface HTableInterface extends Closeable { * @throws IOException if a remote or network exception occurs. */ void setWriteBufferSize(long writeBufferSize) throws IOException; + + /** + * 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), all + * the invocations to the same region server will be batched into one call. The coprocessor + * service is invoked according to the service instance, method name and parameters. + * + * @param methodDescriptor + * the descriptor for the protobuf service method to call. + * @param request + * the method call parameters + * @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 responsePrototype + * the proto type of the response of the method in Service. + * @param + * the response type for the coprocessor Service method + * @throws ServiceException + * @throws Throwable + * @return a map of result values keyed by region name + */ + @InterfaceAudience.Private + Map batchCoprocessorService( + Descriptors.MethodDescriptor methodDescriptor, Message request, + byte[] startKey, byte[] endKey, R responsePrototype) 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), all + * the invocations to the same region server will be batched into one call. The coprocessor + * service is invoked according to the service instance, method name and parameters. + * + *

+ * 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 invocation. + *

+ * + * @param methodDescriptor + * the descriptor for the protobuf service method to call. + * @param request + * the method call parameters + * @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 responsePrototype + * the proto type of the response of the method in Service. + * @param callback + * callback to invoke with the response for each region + * @param + * the response type for the coprocessor Service method + * @throws ServiceException + * @throws Throwable + */ + @InterfaceAudience.Private + void batchCoprocessorService(Descriptors.MethodDescriptor methodDescriptor, + Message request, byte[] startKey, byte[] endKey, R responsePrototype, + Batch.Callback callback) throws ServiceException, Throwable; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java index ba3d76d..d12b903 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java @@ -38,6 +38,8 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.PoolMap; import org.apache.hadoop.hbase.util.PoolMap.PoolType; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; import com.google.protobuf.Service; import com.google.protobuf.ServiceException; @@ -638,5 +640,23 @@ public class HTablePool implements Closeable { byte[] qualifier, long amount, boolean writeToWAL) throws IOException { return table.incrementColumnValue(row, family, qualifier, amount, writeToWAL); } + + @Override + public Map batchCoprocessorService( + Descriptors.MethodDescriptor method, Message request, + byte[] startKey, byte[] endKey, R responsePrototype) throws ServiceException, Throwable { + checkState(); + return table.batchCoprocessorService(method, request, startKey, endKey, + responsePrototype); + } + + @Override + public void batchCoprocessorService( + Descriptors.MethodDescriptor method, Message request, + byte[] startKey, byte[] endKey, R responsePrototype, Callback callback) + throws ServiceException, Throwable { + checkState(); + table.batchCoprocessorService(method, request, startKey, endKey, responsePrototype, callback); + } } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorServiceExec.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorServiceExec.java new file mode 100644 index 0000000..cfc94db --- /dev/null +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorServiceExec.java @@ -0,0 +1,107 @@ +/** + * + * 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.client; + +import com.google.common.base.Objects; +import com.google.protobuf.Descriptors.MethodDescriptor; +import com.google.protobuf.Message; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hbase.client.coprocessor.Batch; +import org.apache.hadoop.hbase.util.Bytes; + + +/** + * Represents a coprocessor service method execution against a single region. While coprocessor + * service calls are performed against a region, this class implements {@link Row} in order to + * make use of the {@link AsyncProcess} framework for batching multi-region calls per region server. + * + *

Note: This class should not be instantiated directly. Use either + * {@link HTable#batchCoprocessorService(MethodDescriptor, Message, byte[], byte[], Batch.Callback, Message)} + * or {@link HTable#batchCoprocessorService(MethodDescriptor, Message, byte[], byte[], Message)} + * instead.

+ */ +@InterfaceAudience.Private +public class RegionCoprocessorServiceExec implements Row { + + /* + * This duplicates region name in MultiAction, but allows us to easily access the region name in + * the AsyncProcessCallback context. + */ + private final byte[] region; + private final byte[] startKey; + private final MethodDescriptor method; + private final Message request; + + public RegionCoprocessorServiceExec(byte[] region, byte[] startKey, + MethodDescriptor method, Message request) { + this.region = region; + this.startKey = startKey; + this.method = method; + this.request = request; + } + + @Override + public byte[] getRow() { + return startKey; + } + + public byte[] getRegion() { + return region; + } + + public MethodDescriptor getMethod() { + return method; + } + + public Message getRequest() { + return request; + } + + @Override + public int compareTo(Row o) { + int res = Bytes.compareTo(this.getRow(), o.getRow()); + if ((o instanceof RegionCoprocessorServiceExec) && res == 0) { + RegionCoprocessorServiceExec exec = (RegionCoprocessorServiceExec) o; + res = method.getFullName().compareTo(exec.getMethod().getFullName()); + if (res == 0) { + res = Bytes.compareTo(request.toByteArray(), exec.getRequest().toByteArray()); + } + } + return res; + } + + @Override + public int hashCode() { + return Objects.hashCode(Bytes.hashCode(this.getRow()), method.getFullName(), request); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + Row other = (Row) obj; + return compareTo(other) == 0; + } +} diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java index 37f01bf..49a7a44 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java @@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.RegionCoprocessorServiceExec; import org.apache.hadoop.hbase.client.Row; import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.Scan; @@ -533,6 +534,14 @@ public final class RequestConverter { } else if (row instanceof Increment) { regionActionBuilder.addAction(actionBuilder.setMutation( ProtobufUtil.toMutation((Increment)row, mutationBuilder, action.getNonce()))); + } else if (row instanceof RegionCoprocessorServiceExec) { + RegionCoprocessorServiceExec exec = (RegionCoprocessorServiceExec) row; + regionActionBuilder.addAction(actionBuilder.setServiceCall( + ClientProtos.CoprocessorServiceCall.newBuilder() + .setRow(HBaseZeroCopyByteString.wrap(exec.getRow())) + .setServiceName(exec.getMethod().getService().getFullName()) + .setMethodName(exec.getMethod().getName()) + .setRequest(exec.getRequest().toByteString()))); } else if (row instanceof RowMutations) { throw new UnsupportedOperationException("No RowMutations in multi calls; use mutateRow"); } else { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java index c854a99..26b52c8 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java @@ -122,6 +122,9 @@ public final class ResponseConverter { } else if (roe.hasResult()) { results.add(regionName, new Pair(roe.getIndex(), ProtobufUtil.toResult(roe.getResult(), cells))); + } else if (roe.hasServiceResult()) { + results.add(regionName, new Pair(roe.getIndex(), + roe.getServiceResult())); } else { // no result & no exception. Unexpected. throw new IllegalStateException("No result & no exception roe=" + roe + diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java index 9e1952d..fd81fe3 100644 --- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java +++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java @@ -21316,6 +21316,563 @@ public final class ClientProtos { // @@protoc_insertion_point(class_scope:CoprocessorServiceCall) } + public interface CoprocessorServiceResultOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // optional .NameBytesPair value = 1; + /** + * optional .NameBytesPair value = 1; + */ + boolean hasValue(); + /** + * optional .NameBytesPair value = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getValue(); + /** + * optional .NameBytesPair value = 1; + */ + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getValueOrBuilder(); + } + /** + * Protobuf type {@code CoprocessorServiceResult} + */ + public static final class CoprocessorServiceResult extends + com.google.protobuf.GeneratedMessage + implements CoprocessorServiceResultOrBuilder { + // Use CoprocessorServiceResult.newBuilder() to construct. + private CoprocessorServiceResult(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private CoprocessorServiceResult(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final CoprocessorServiceResult defaultInstance; + public static CoprocessorServiceResult getDefaultInstance() { + return defaultInstance; + } + + public CoprocessorServiceResult getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private CoprocessorServiceResult( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder subBuilder = null; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + subBuilder = value_.toBuilder(); + } + value_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(value_); + value_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000001; + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_CoprocessorServiceResult_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_CoprocessorServiceResult_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public CoprocessorServiceResult parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new CoprocessorServiceResult(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // optional .NameBytesPair value = 1; + public static final int VALUE_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value_; + /** + * optional .NameBytesPair value = 1; + */ + public boolean hasValue() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .NameBytesPair value = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getValue() { + return value_; + } + /** + * optional .NameBytesPair value = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getValueOrBuilder() { + return value_; + } + + private void initFields() { + 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 (hasValue()) { + 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, 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, 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.CoprocessorServiceResult)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult other = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult) obj; + + boolean result = true; + result = result && (hasValue() == other.hasValue()); + if (hasValue()) { + result = result && getValue() + .equals(other.getValue()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasValue()) { + hash = (37 * hash) + VALUE_FIELD_NUMBER; + hash = (53 * hash) + getValue().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult 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; + } + /** + * Protobuf type {@code CoprocessorServiceResult} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResultOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_CoprocessorServiceResult_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.internal_static_CoprocessorServiceResult_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.class, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getValueFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (valueBuilder_ == null) { + value_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); + } else { + valueBuilder_.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.ClientProtos.internal_static_CoprocessorServiceResult_descriptor; + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult build() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + 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.CoprocessorServiceResult) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.getDefaultInstance()) return this; + if (other.hasValue()) { + mergeValue(other.getValue()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (hasValue()) { + if (!getValue().isInitialized()) { + + return false; + } + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // optional .NameBytesPair value = 1; + 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_; + /** + * optional .NameBytesPair value = 1; + */ + public boolean hasValue() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional .NameBytesPair value = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getValue() { + if (valueBuilder_ == null) { + return value_; + } else { + return valueBuilder_.getMessage(); + } + } + /** + * optional .NameBytesPair value = 1; + */ + 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_ |= 0x00000001; + return this; + } + /** + * optional .NameBytesPair value = 1; + */ + 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_ |= 0x00000001; + return this; + } + /** + * optional .NameBytesPair value = 1; + */ + public Builder mergeValue(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair value) { + if (valueBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + 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_ |= 0x00000001; + return this; + } + /** + * optional .NameBytesPair value = 1; + */ + public Builder clearValue() { + if (valueBuilder_ == null) { + value_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); + onChanged(); + } else { + valueBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + /** + * optional .NameBytesPair value = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder getValueBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getValueFieldBuilder().getBuilder(); + } + /** + * optional .NameBytesPair value = 1; + */ + public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getValueOrBuilder() { + if (valueBuilder_ != null) { + return valueBuilder_.getMessageOrBuilder(); + } else { + return value_; + } + } + /** + * optional .NameBytesPair value = 1; + */ + 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:CoprocessorServiceResult) + } + + static { + defaultInstance = new CoprocessorServiceResult(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:CoprocessorServiceResult) + } + public interface CoprocessorServiceRequestOrBuilder extends com.google.protobuf.MessageOrBuilder { @@ -22922,6 +23479,20 @@ public final class ClientProtos { * optional .Get get = 3; */ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetOrBuilder getGetOrBuilder(); + + // optional .CoprocessorServiceCall service_call = 4; + /** + * optional .CoprocessorServiceCall service_call = 4; + */ + boolean hasServiceCall(); + /** + * optional .CoprocessorServiceCall service_call = 4; + */ + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall getServiceCall(); + /** + * optional .CoprocessorServiceCall service_call = 4; + */ + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder getServiceCallOrBuilder(); } /** * Protobuf type {@code Action} @@ -23009,6 +23580,19 @@ public final class ClientProtos { bitField0_ |= 0x00000004; break; } + case 34: { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.Builder subBuilder = null; + if (((bitField0_ & 0x00000008) == 0x00000008)) { + subBuilder = serviceCall_.toBuilder(); + } + serviceCall_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(serviceCall_); + serviceCall_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000008; + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -23119,10 +23703,33 @@ public final class ClientProtos { return get_; } + // optional .CoprocessorServiceCall service_call = 4; + public static final int SERVICE_CALL_FIELD_NUMBER = 4; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall serviceCall_; + /** + * optional .CoprocessorServiceCall service_call = 4; + */ + public boolean hasServiceCall() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional .CoprocessorServiceCall service_call = 4; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall getServiceCall() { + return serviceCall_; + } + /** + * optional .CoprocessorServiceCall service_call = 4; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder getServiceCallOrBuilder() { + return serviceCall_; + } + private void initFields() { index_ = 0; mutation_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance(); get_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Get.getDefaultInstance(); + serviceCall_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.getDefaultInstance(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -23141,6 +23748,12 @@ public final class ClientProtos { return false; } } + if (hasServiceCall()) { + if (!getServiceCall().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } memoizedIsInitialized = 1; return true; } @@ -23157,6 +23770,9 @@ public final class ClientProtos { if (((bitField0_ & 0x00000004) == 0x00000004)) { output.writeMessage(3, get_); } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeMessage(4, serviceCall_); + } getUnknownFields().writeTo(output); } @@ -23178,6 +23794,10 @@ public final class ClientProtos { size += com.google.protobuf.CodedOutputStream .computeMessageSize(3, get_); } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(4, serviceCall_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -23216,6 +23836,11 @@ public final class ClientProtos { result = result && getGet() .equals(other.getGet()); } + result = result && (hasServiceCall() == other.hasServiceCall()); + if (hasServiceCall()) { + result = result && getServiceCall() + .equals(other.getServiceCall()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -23241,6 +23866,10 @@ public final class ClientProtos { hash = (37 * hash) + GET_FIELD_NUMBER; hash = (53 * hash) + getGet().hashCode(); } + if (hasServiceCall()) { + hash = (37 * hash) + SERVICE_CALL_FIELD_NUMBER; + hash = (53 * hash) + getServiceCall().hashCode(); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -23348,6 +23977,7 @@ public final class ClientProtos { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { getMutationFieldBuilder(); getGetFieldBuilder(); + getServiceCallFieldBuilder(); } } private static Builder create() { @@ -23370,6 +24000,12 @@ public final class ClientProtos { getBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000004); + if (serviceCallBuilder_ == null) { + serviceCall_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.getDefaultInstance(); + } else { + serviceCallBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -23416,7 +24052,15 @@ public final class ClientProtos { if (getBuilder_ == null) { result.get_ = get_; } else { - result.get_ = getBuilder_.build(); + result.get_ = getBuilder_.build(); + } + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + if (serviceCallBuilder_ == null) { + result.serviceCall_ = serviceCall_; + } else { + result.serviceCall_ = serviceCallBuilder_.build(); } result.bitField0_ = to_bitField0_; onBuilt(); @@ -23443,6 +24087,9 @@ public final class ClientProtos { if (other.hasGet()) { mergeGet(other.getGet()); } + if (other.hasServiceCall()) { + mergeServiceCall(other.getServiceCall()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -23460,6 +24107,12 @@ public final class ClientProtos { return false; } } + if (hasServiceCall()) { + if (!getServiceCall().isInitialized()) { + + return false; + } + } return true; } @@ -23769,6 +24422,123 @@ public final class ClientProtos { return getBuilder_; } + // optional .CoprocessorServiceCall service_call = 4; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall serviceCall_ = 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> serviceCallBuilder_; + /** + * optional .CoprocessorServiceCall service_call = 4; + */ + public boolean hasServiceCall() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional .CoprocessorServiceCall service_call = 4; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall getServiceCall() { + if (serviceCallBuilder_ == null) { + return serviceCall_; + } else { + return serviceCallBuilder_.getMessage(); + } + } + /** + * optional .CoprocessorServiceCall service_call = 4; + */ + public Builder setServiceCall(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall value) { + if (serviceCallBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + serviceCall_ = value; + onChanged(); + } else { + serviceCallBuilder_.setMessage(value); + } + bitField0_ |= 0x00000008; + return this; + } + /** + * optional .CoprocessorServiceCall service_call = 4; + */ + public Builder setServiceCall( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.Builder builderForValue) { + if (serviceCallBuilder_ == null) { + serviceCall_ = builderForValue.build(); + onChanged(); + } else { + serviceCallBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000008; + return this; + } + /** + * optional .CoprocessorServiceCall service_call = 4; + */ + public Builder mergeServiceCall(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall value) { + if (serviceCallBuilder_ == null) { + if (((bitField0_ & 0x00000008) == 0x00000008) && + serviceCall_ != org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.getDefaultInstance()) { + serviceCall_ = + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.newBuilder(serviceCall_).mergeFrom(value).buildPartial(); + } else { + serviceCall_ = value; + } + onChanged(); + } else { + serviceCallBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000008; + return this; + } + /** + * optional .CoprocessorServiceCall service_call = 4; + */ + public Builder clearServiceCall() { + if (serviceCallBuilder_ == null) { + serviceCall_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.getDefaultInstance(); + onChanged(); + } else { + serviceCallBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + /** + * optional .CoprocessorServiceCall service_call = 4; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.Builder getServiceCallBuilder() { + bitField0_ |= 0x00000008; + onChanged(); + return getServiceCallFieldBuilder().getBuilder(); + } + /** + * optional .CoprocessorServiceCall service_call = 4; + */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder getServiceCallOrBuilder() { + if (serviceCallBuilder_ != null) { + return serviceCallBuilder_.getMessageOrBuilder(); + } else { + return serviceCall_; + } + } + /** + * optional .CoprocessorServiceCall service_call = 4; + */ + 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> + getServiceCallFieldBuilder() { + if (serviceCallBuilder_ == null) { + serviceCallBuilder_ = 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>( + serviceCall_, + getParentForChildren(), + isClean()); + serviceCall_ = null; + } + return serviceCallBuilder_; + } + // @@protoc_insertion_point(builder_scope:Action) } @@ -24903,6 +25673,32 @@ public final class ClientProtos { * optional .NameBytesPair exception = 3; */ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getExceptionOrBuilder(); + + // optional .CoprocessorServiceResult service_result = 4; + /** + * optional .CoprocessorServiceResult service_result = 4; + * + *
+     * result if this was a coprocessor service call
+     * 
+ */ + boolean hasServiceResult(); + /** + * optional .CoprocessorServiceResult service_result = 4; + * + *
+     * result if this was a coprocessor service call
+     * 
+ */ + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult getServiceResult(); + /** + * optional .CoprocessorServiceResult service_result = 4; + * + *
+     * result if this was a coprocessor service call
+     * 
+ */ + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResultOrBuilder getServiceResultOrBuilder(); } /** * Protobuf type {@code ResultOrException} @@ -24993,6 +25789,19 @@ public final class ClientProtos { bitField0_ |= 0x00000004; break; } + case 34: { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.Builder subBuilder = null; + if (((bitField0_ & 0x00000008) == 0x00000008)) { + subBuilder = serviceResult_.toBuilder(); + } + serviceResult_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.PARSER, extensionRegistry); + if (subBuilder != null) { + subBuilder.mergeFrom(serviceResult_); + serviceResult_ = subBuilder.buildPartial(); + } + bitField0_ |= 0x00000008; + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -25103,10 +25912,45 @@ public final class ClientProtos { return exception_; } + // optional .CoprocessorServiceResult service_result = 4; + public static final int SERVICE_RESULT_FIELD_NUMBER = 4; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult serviceResult_; + /** + * optional .CoprocessorServiceResult service_result = 4; + * + *
+     * result if this was a coprocessor service call
+     * 
+ */ + public boolean hasServiceResult() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional .CoprocessorServiceResult service_result = 4; + * + *
+     * result if this was a coprocessor service call
+     * 
+ */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult getServiceResult() { + return serviceResult_; + } + /** + * optional .CoprocessorServiceResult service_result = 4; + * + *
+     * result if this was a coprocessor service call
+     * 
+ */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResultOrBuilder getServiceResultOrBuilder() { + return serviceResult_; + } + private void initFields() { index_ = 0; result_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Result.getDefaultInstance(); exception_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.getDefaultInstance(); + serviceResult_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.getDefaultInstance(); } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -25119,6 +25963,12 @@ public final class ClientProtos { return false; } } + if (hasServiceResult()) { + if (!getServiceResult().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } memoizedIsInitialized = 1; return true; } @@ -25135,6 +25985,9 @@ public final class ClientProtos { if (((bitField0_ & 0x00000004) == 0x00000004)) { output.writeMessage(3, exception_); } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeMessage(4, serviceResult_); + } getUnknownFields().writeTo(output); } @@ -25156,6 +26009,10 @@ public final class ClientProtos { size += com.google.protobuf.CodedOutputStream .computeMessageSize(3, exception_); } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(4, serviceResult_); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -25194,6 +26051,11 @@ public final class ClientProtos { result = result && getException() .equals(other.getException()); } + result = result && (hasServiceResult() == other.hasServiceResult()); + if (hasServiceResult()) { + result = result && getServiceResult() + .equals(other.getServiceResult()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -25219,6 +26081,10 @@ public final class ClientProtos { hash = (37 * hash) + EXCEPTION_FIELD_NUMBER; hash = (53 * hash) + getException().hashCode(); } + if (hasServiceResult()) { + hash = (37 * hash) + SERVICE_RESULT_FIELD_NUMBER; + hash = (53 * hash) + getServiceResult().hashCode(); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -25329,6 +26195,7 @@ public final class ClientProtos { if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { getResultFieldBuilder(); getExceptionFieldBuilder(); + getServiceResultFieldBuilder(); } } private static Builder create() { @@ -25351,6 +26218,12 @@ public final class ClientProtos { exceptionBuilder_.clear(); } bitField0_ = (bitField0_ & ~0x00000004); + if (serviceResultBuilder_ == null) { + serviceResult_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.getDefaultInstance(); + } else { + serviceResultBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -25399,6 +26272,14 @@ public final class ClientProtos { } else { result.exception_ = exceptionBuilder_.build(); } + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + if (serviceResultBuilder_ == null) { + result.serviceResult_ = serviceResult_; + } else { + result.serviceResult_ = serviceResultBuilder_.build(); + } result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -25424,6 +26305,9 @@ public final class ClientProtos { if (other.hasException()) { mergeException(other.getException()); } + if (other.hasServiceResult()) { + mergeServiceResult(other.getServiceResult()); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -25435,6 +26319,12 @@ public final class ClientProtos { return false; } } + if (hasServiceResult()) { + if (!getServiceResult().isInitialized()) { + + return false; + } + } return true; } @@ -25744,6 +26634,159 @@ public final class ClientProtos { return exceptionBuilder_; } + // optional .CoprocessorServiceResult service_result = 4; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult serviceResult_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResultOrBuilder> serviceResultBuilder_; + /** + * optional .CoprocessorServiceResult service_result = 4; + * + *
+       * result if this was a coprocessor service call
+       * 
+ */ + public boolean hasServiceResult() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * optional .CoprocessorServiceResult service_result = 4; + * + *
+       * result if this was a coprocessor service call
+       * 
+ */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult getServiceResult() { + if (serviceResultBuilder_ == null) { + return serviceResult_; + } else { + return serviceResultBuilder_.getMessage(); + } + } + /** + * optional .CoprocessorServiceResult service_result = 4; + * + *
+       * result if this was a coprocessor service call
+       * 
+ */ + public Builder setServiceResult(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult value) { + if (serviceResultBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + serviceResult_ = value; + onChanged(); + } else { + serviceResultBuilder_.setMessage(value); + } + bitField0_ |= 0x00000008; + return this; + } + /** + * optional .CoprocessorServiceResult service_result = 4; + * + *
+       * result if this was a coprocessor service call
+       * 
+ */ + public Builder setServiceResult( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.Builder builderForValue) { + if (serviceResultBuilder_ == null) { + serviceResult_ = builderForValue.build(); + onChanged(); + } else { + serviceResultBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000008; + return this; + } + /** + * optional .CoprocessorServiceResult service_result = 4; + * + *
+       * result if this was a coprocessor service call
+       * 
+ */ + public Builder mergeServiceResult(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult value) { + if (serviceResultBuilder_ == null) { + if (((bitField0_ & 0x00000008) == 0x00000008) && + serviceResult_ != org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.getDefaultInstance()) { + serviceResult_ = + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.newBuilder(serviceResult_).mergeFrom(value).buildPartial(); + } else { + serviceResult_ = value; + } + onChanged(); + } else { + serviceResultBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000008; + return this; + } + /** + * optional .CoprocessorServiceResult service_result = 4; + * + *
+       * result if this was a coprocessor service call
+       * 
+ */ + public Builder clearServiceResult() { + if (serviceResultBuilder_ == null) { + serviceResult_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.getDefaultInstance(); + onChanged(); + } else { + serviceResultBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + /** + * optional .CoprocessorServiceResult service_result = 4; + * + *
+       * result if this was a coprocessor service call
+       * 
+ */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.Builder getServiceResultBuilder() { + bitField0_ |= 0x00000008; + onChanged(); + return getServiceResultFieldBuilder().getBuilder(); + } + /** + * optional .CoprocessorServiceResult service_result = 4; + * + *
+       * result if this was a coprocessor service call
+       * 
+ */ + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResultOrBuilder getServiceResultOrBuilder() { + if (serviceResultBuilder_ != null) { + return serviceResultBuilder_.getMessageOrBuilder(); + } else { + return serviceResult_; + } + } + /** + * optional .CoprocessorServiceResult service_result = 4; + * + *
+       * result if this was a coprocessor service call
+       * 
+ */ + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResultOrBuilder> + getServiceResultFieldBuilder() { + if (serviceResultBuilder_ == null) { + serviceResultBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResultOrBuilder>( + serviceResult_, + getParentForChildren(), + isClean()); + serviceResult_ = null; + } + return serviceResultBuilder_; + } + // @@protoc_insertion_point(builder_scope:ResultOrException) } @@ -29003,6 +30046,11 @@ public final class ClientProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_CoprocessorServiceCall_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor + internal_static_CoprocessorServiceResult_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_CoprocessorServiceResult_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor internal_static_CoprocessorServiceRequest_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable @@ -29121,34 +30169,38 @@ public final class ClientProtos { "\025BulkLoadHFileResponse\022\016\n\006loaded\030\001 \002(\010\"a" + "\n\026CoprocessorServiceCall\022\013\n\003row\030\001 \002(\014\022\024\n" + "\014service_name\030\002 \002(\t\022\023\n\013method_name\030\003 \002(\t", - "\022\017\n\007request\030\004 \002(\014\"d\n\031CoprocessorServiceR" + - "equest\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier" + - "\022%\n\004call\030\002 \002(\0132\027.CoprocessorServiceCall\"" + - "]\n\032CoprocessorServiceResponse\022 \n\006region\030" + - "\001 \002(\0132\020.RegionSpecifier\022\035\n\005value\030\002 \002(\0132\016" + - ".NameBytesPair\"L\n\006Action\022\r\n\005index\030\001 \001(\r\022" + - " \n\010mutation\030\002 \001(\0132\016.MutationProto\022\021\n\003get" + - "\030\003 \001(\0132\004.Get\"Y\n\014RegionAction\022 \n\006region\030\001" + - " \002(\0132\020.RegionSpecifier\022\016\n\006atomic\030\002 \001(\010\022\027" + - "\n\006action\030\003 \003(\0132\007.Action\"^\n\021ResultOrExcep", - "tion\022\r\n\005index\030\001 \001(\r\022\027\n\006result\030\002 \001(\0132\007.Re" + - "sult\022!\n\texception\030\003 \001(\0132\016.NameBytesPair\"" + - "f\n\022RegionActionResult\022-\n\021resultOrExcepti" + - "on\030\001 \003(\0132\022.ResultOrException\022!\n\texceptio" + - "n\030\002 \001(\0132\016.NameBytesPair\"G\n\014MultiRequest\022" + - "#\n\014regionAction\030\001 \003(\0132\r.RegionAction\022\022\n\n" + - "nonceGroup\030\002 \001(\004\"@\n\rMultiResponse\022/\n\022reg" + - "ionActionResult\030\001 \003(\0132\023.RegionActionResu" + - "lt2\261\002\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\rBulkLoadHFile\022\025.BulkLo" + - "adHFileRequest\032\026.BulkLoadHFileResponse\022F" + - "\n\013ExecService\022\032.CoprocessorServiceReques" + - "t\032\033.CoprocessorServiceResponse\022&\n\005Multi\022" + - "\r.MultiRequest\032\016.MultiResponseBB\n*org.ap" + - "ache.hadoop.hbase.protobuf.generatedB\014Cl" + - "ientProtosH\001\210\001\001\240\001\001" + "\022\017\n\007request\030\004 \002(\014\"9\n\030CoprocessorServiceR" + + "esult\022\035\n\005value\030\001 \001(\0132\016.NameBytesPair\"d\n\031" + + "CoprocessorServiceRequest\022 \n\006region\030\001 \002(" + + "\0132\020.RegionSpecifier\022%\n\004call\030\002 \002(\0132\027.Copr" + + "ocessorServiceCall\"]\n\032CoprocessorService" + + "Response\022 \n\006region\030\001 \002(\0132\020.RegionSpecifi" + + "er\022\035\n\005value\030\002 \002(\0132\016.NameBytesPair\"{\n\006Act" + + "ion\022\r\n\005index\030\001 \001(\r\022 \n\010mutation\030\002 \001(\0132\016.M" + + "utationProto\022\021\n\003get\030\003 \001(\0132\004.Get\022-\n\014servi" + + "ce_call\030\004 \001(\0132\027.CoprocessorServiceCall\"Y", + "\n\014RegionAction\022 \n\006region\030\001 \002(\0132\020.RegionS" + + "pecifier\022\016\n\006atomic\030\002 \001(\010\022\027\n\006action\030\003 \003(\013" + + "2\007.Action\"\221\001\n\021ResultOrException\022\r\n\005index" + + "\030\001 \001(\r\022\027\n\006result\030\002 \001(\0132\007.Result\022!\n\texcep" + + "tion\030\003 \001(\0132\016.NameBytesPair\0221\n\016service_re" + + "sult\030\004 \001(\0132\031.CoprocessorServiceResult\"f\n" + + "\022RegionActionResult\022-\n\021resultOrException" + + "\030\001 \003(\0132\022.ResultOrException\022!\n\texception\030" + + "\002 \001(\0132\016.NameBytesPair\"G\n\014MultiRequest\022#\n" + + "\014regionAction\030\001 \003(\0132\r.RegionAction\022\022\n\nno", + "nceGroup\030\002 \001(\004\"@\n\rMultiResponse\022/\n\022regio" + + "nActionResult\030\001 \003(\0132\023.RegionActionResult" + + "2\261\002\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\rBulkLoadHFile\022\025.BulkLoad" + + "HFileRequest\032\026.BulkLoadHFileResponse\022F\n\013" + + "ExecService\022\032.CoprocessorServiceRequest\032" + + "\033.CoprocessorServiceResponse\022&\n\005Multi\022\r." + + "MultiRequest\032\016.MultiResponseBB\n*org.apac", + "he.hadoop.hbase.protobuf.generatedB\014Clie" + + "ntProtosH\001\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -29275,50 +30327,56 @@ public final class ClientProtos { com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_CoprocessorServiceCall_descriptor, new java.lang.String[] { "Row", "ServiceName", "MethodName", "Request", }); - internal_static_CoprocessorServiceRequest_descriptor = + internal_static_CoprocessorServiceResult_descriptor = getDescriptor().getMessageTypes().get(17); + internal_static_CoprocessorServiceResult_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_CoprocessorServiceResult_descriptor, + new java.lang.String[] { "Value", }); + internal_static_CoprocessorServiceRequest_descriptor = + getDescriptor().getMessageTypes().get(18); internal_static_CoprocessorServiceRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_CoprocessorServiceRequest_descriptor, new java.lang.String[] { "Region", "Call", }); internal_static_CoprocessorServiceResponse_descriptor = - getDescriptor().getMessageTypes().get(18); + getDescriptor().getMessageTypes().get(19); internal_static_CoprocessorServiceResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_CoprocessorServiceResponse_descriptor, new java.lang.String[] { "Region", "Value", }); internal_static_Action_descriptor = - getDescriptor().getMessageTypes().get(19); + getDescriptor().getMessageTypes().get(20); internal_static_Action_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_Action_descriptor, - new java.lang.String[] { "Index", "Mutation", "Get", }); + new java.lang.String[] { "Index", "Mutation", "Get", "ServiceCall", }); internal_static_RegionAction_descriptor = - getDescriptor().getMessageTypes().get(20); + getDescriptor().getMessageTypes().get(21); internal_static_RegionAction_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RegionAction_descriptor, new java.lang.String[] { "Region", "Atomic", "Action", }); internal_static_ResultOrException_descriptor = - getDescriptor().getMessageTypes().get(21); + getDescriptor().getMessageTypes().get(22); internal_static_ResultOrException_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ResultOrException_descriptor, - new java.lang.String[] { "Index", "Result", "Exception", }); + new java.lang.String[] { "Index", "Result", "Exception", "ServiceResult", }); internal_static_RegionActionResult_descriptor = - getDescriptor().getMessageTypes().get(22); + getDescriptor().getMessageTypes().get(23); internal_static_RegionActionResult_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_RegionActionResult_descriptor, new java.lang.String[] { "ResultOrException", "Exception", }); internal_static_MultiRequest_descriptor = - getDescriptor().getMessageTypes().get(23); + getDescriptor().getMessageTypes().get(24); internal_static_MultiRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_MultiRequest_descriptor, new java.lang.String[] { "RegionAction", "NonceGroup", }); internal_static_MultiResponse_descriptor = - getDescriptor().getMessageTypes().get(24); + getDescriptor().getMessageTypes().get(25); internal_static_MultiResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_MultiResponse_descriptor, diff --git a/hbase-protocol/src/main/protobuf/Client.proto b/hbase-protocol/src/main/protobuf/Client.proto index 180b711..631d0cd 100644 --- a/hbase-protocol/src/main/protobuf/Client.proto +++ b/hbase-protocol/src/main/protobuf/Client.proto @@ -303,6 +303,10 @@ message CoprocessorServiceCall { required bytes request = 4; } +message CoprocessorServiceResult { + optional NameBytesPair value = 1; +} + message CoprocessorServiceRequest { required RegionSpecifier region = 1; required CoprocessorServiceCall call = 2; @@ -320,6 +324,7 @@ message Action { optional uint32 index = 1; optional MutationProto mutation = 2; optional Get get = 3; + optional CoprocessorServiceCall service_call = 4; } /** @@ -343,6 +348,8 @@ message ResultOrException { optional uint32 index = 1; optional Result result = 2; optional NameBytesPair exception = 3; + // result if this was a coprocessor service call + optional CoprocessorServiceResult service_result = 4; } /** 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 032ad36..7595b04 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 @@ -33,6 +33,7 @@ import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicInteger; +import com.google.protobuf.Descriptors; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; @@ -62,6 +63,7 @@ import org.apache.hadoop.hbase.client.Row; import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.coprocessor.Batch; +import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.util.Bytes; @@ -70,6 +72,8 @@ import org.apache.hadoop.hbase.util.SortedCopyOnWriteSet; import org.apache.hadoop.hbase.util.VersionInfo; import org.apache.hadoop.io.MultipleIOException; +import com.google.protobuf.Descriptors.ServiceDescriptor; +import com.google.protobuf.Message; import com.google.protobuf.Service; import com.google.protobuf.ServiceException; @@ -622,6 +626,21 @@ public abstract class CoprocessorHost { byte[] qualifier, long amount, boolean writeToWAL) throws IOException { return table.incrementColumnValue(row, family, qualifier, amount, writeToWAL); } + + @Override + public Map batchCoprocessorService( + Descriptors.MethodDescriptor method, Message request, byte[] startKey, + byte[] endKey, R responsePrototype) throws ServiceException, Throwable { + return table.batchCoprocessorService(method, request, startKey, endKey, responsePrototype); + } + + @Override + public void batchCoprocessorService(Descriptors.MethodDescriptor method, + Message request, byte[] startKey, byte[] endKey, R responsePrototype, + Callback callback) throws ServiceException, Throwable { + table.batchCoprocessorService(method, request, startKey, endKey, responsePrototype, + callback); + } } /** The coprocessor */ 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 039ced7..0f9fe8d 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 @@ -3281,12 +3281,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa checkOpen(); requestCount.increment(); 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(); - } + Message result = execServiceOnRegion(region, request.getCall()); CoprocessorServiceResponse.Builder builder = CoprocessorServiceResponse.newBuilder(); builder.setRegion(RequestConverter.buildRegionSpecifier( @@ -3300,6 +3295,17 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa } } + private Message execServiceOnRegion(HRegion region, final ClientProtos.CoprocessorServiceCall serviceCall) + throws IOException { + // ignore the passed in controller (from the serialized call) + ServerRpcController execController = new ServerRpcController(); + Message result = region.execService(execController, serviceCall); + if (execController.getFailedOn() != null) { + throw execController.getFailedOn(); + } + return result; + } + /** * Execute multiple actions on a table: get, mutate, and/or execCoprocessor * @@ -3390,6 +3396,20 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa if (action.hasGet()) { Get get = ProtobufUtil.toGet(action.getGet()); r = region.get(get); + } else if (action.hasServiceCall()) { + resultOrExceptionBuilder = ResultOrException.newBuilder(); + try { + Message result = execServiceOnRegion(region, action.getServiceCall()); + ClientProtos.CoprocessorServiceResult.Builder serviceResultBuilder = + ClientProtos.CoprocessorServiceResult.newBuilder(); + resultOrExceptionBuilder.setServiceResult( + serviceResultBuilder.setValue( + serviceResultBuilder.getValueBuilder() + .setName(result.getClass().getName()) + .setValue(result.toByteString()))); + } catch (IOException ioe) { + resultOrExceptionBuilder.setException(ResponseConverter.buildException(ioe)); + } } else if (action.hasMutation()) { MutationType type = action.getMutation().getMutateType(); if (type != MutationType.PUT && type != MutationType.DELETE && mutations != null && @@ -4330,7 +4350,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa case DELETE: rm.add(ProtobufUtil.toDelete(action.getMutation(), cellScanner)); break; - default: + default: throw new DoNotRetryIOException("Atomic put and/or delete only, not " + type.name()); } } 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 0e17362..e79e0e8 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 @@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.client.Row; import org.apache.hadoop.hbase.client.RowMutations; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.coprocessor.Batch; +import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.io.TimeRange; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; @@ -65,6 +66,8 @@ import org.apache.hadoop.hbase.rest.model.TableSchemaModel; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.util.StringUtils; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; import com.google.protobuf.Service; import com.google.protobuf.ServiceException; @@ -808,4 +811,19 @@ public class RemoteHTable implements HTableInterface { long amount, boolean writeToWAL) throws IOException { throw new IOException("incrementColumnValue not supported"); } + + @Override + public Map batchCoprocessorService( + Descriptors.MethodDescriptor method, Message request, + byte[] startKey, byte[] endKey, R responsePrototype) throws ServiceException, Throwable { + throw new UnsupportedOperationException("batchCoprocessorService not implemented"); + } + + @Override + public void batchCoprocessorService( + Descriptors.MethodDescriptor method, Message request, + byte[] startKey, byte[] endKey, R responsePrototype, Callback callback) + throws ServiceException, Throwable { + throw new UnsupportedOperationException("batchCoprocessorService not implemented"); + } } 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 1cd5e52..8e37470 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 @@ -112,6 +112,7 @@ implements Coprocessor, CoprocessorService { } } } + LOG.info("Returning result " + sumResult); done.run(SumResponse.newBuilder().setSum(sumResult).build()); } } \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointNullResponse.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointNullResponse.java new file mode 100644 index 0000000..c7a85b3 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointNullResponse.java @@ -0,0 +1,125 @@ +/** + * 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 java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.ColumnAggregationServiceNullResponse; +import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest; +import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse; +import org.apache.hadoop.hbase.protobuf.ResponseConverter; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.util.Bytes; + +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; + +/** + * Test coprocessor endpoint that always returns {@code null} for requests to the last region in the table. This + * allows tests to provide assurance of correct {@code null} handling for response values. + */ +public class ColumnAggregationEndpointNullResponse + extends + ColumnAggregationServiceNullResponse +implements Coprocessor, CoprocessorService { + static final Log LOG = LogFactory.getLog(ColumnAggregationEndpointNullResponse.class); + private RegionCoprocessorEnvironment env = null; + @Override + public Service getService() { + return this; + } + + @Override + public void start(CoprocessorEnvironment env) throws IOException { + if (env instanceof RegionCoprocessorEnvironment) { + this.env = (RegionCoprocessorEnvironment)env; + return; + } + throw new CoprocessorException("Must be loaded on a table region!"); + } + + @Override + public void stop(CoprocessorEnvironment env) throws IOException { + // Nothing to do. + } + + @Override + public void sum(RpcController controller, SumRequest request, RpcCallback done) { + // aggregate at each region + Scan scan = new Scan(); + // Family is required in pb. Qualifier is not. + byte[] family = request.getFamily().toByteArray(); + byte[] qualifier = request.hasQualifier() ? request.getQualifier().toByteArray() : null; + if (request.hasQualifier()) { + scan.addColumn(family, qualifier); + } else { + scan.addFamily(family); + } + int sumResult = 0; + InternalScanner scanner = null; + try { + HRegion region = this.env.getRegion(); + // for the last region in the table, return null to test null handling + if (Bytes.equals(region.getEndKey(), HConstants.EMPTY_END_ROW)) { + done.run(null); + return; + } + scanner = region.getScanner(scan); + List curVals = new ArrayList(); + boolean hasMore = false; + do { + curVals.clear(); + hasMore = scanner.next(curVals); + for (Cell kv : curVals) { + if (CellUtil.matchingQualifier(kv, qualifier)) { + sumResult += Bytes.toInt(kv.getValueArray(), kv.getValueOffset()); + } + } + } while (hasMore); + } catch (IOException e) { + ResponseConverter.setControllerException(controller, e); + // Set result to -1 to indicate error. + sumResult = -1; + LOG.info("Setting sum result to -1 to indicate error", e); + } finally { + if (scanner != null) { + try { + scanner.close(); + } catch (IOException e) { + ResponseConverter.setControllerException(controller, e); + sumResult = -1; + LOG.info("Setting sum result to -1 to indicate error", e); + } + } + } + done.run(SumResponse.newBuilder().setSum(sumResult).build()); + LOG.info("Returning sum " + sumResult + " for region " + Bytes.toStringBinary(env.getRegion().getRegionName())); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointWithErrors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointWithErrors.java new file mode 100644 index 0000000..733e83a --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointWithErrors.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.coprocessor; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos; +import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest; +import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse; +import org.apache.hadoop.hbase.protobuf.ResponseConverter; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.util.Bytes; + +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; + +/** + * Test coprocessor endpoint that always throws a {@link DoNotRetryIOException} for requests on the last region + * in the table. This allows tests to ensure correct error handling of coprocessor endpoints throwing exceptions. + */ +public class ColumnAggregationEndpointWithErrors + extends + ColumnAggregationWithErrorsProtos.ColumnAggregationServiceWithErrors +implements Coprocessor, CoprocessorService { + static final Log LOG = LogFactory.getLog(ColumnAggregationEndpointWithErrors.class); + private RegionCoprocessorEnvironment env = null; + @Override + public Service getService() { + return this; + } + + @Override + public void start(CoprocessorEnvironment env) throws IOException { + if (env instanceof RegionCoprocessorEnvironment) { + this.env = (RegionCoprocessorEnvironment)env; + return; + } + throw new CoprocessorException("Must be loaded on a table region!"); + } + + @Override + public void stop(CoprocessorEnvironment env) throws IOException { + // Nothing to do. + } + + @Override + public void sum(RpcController controller, SumRequest request, RpcCallback done) { + // aggregate at each region + Scan scan = new Scan(); + // Family is required in pb. Qualifier is not. + byte[] family = request.getFamily().toByteArray(); + byte[] qualifier = request.hasQualifier() ? request.getQualifier().toByteArray() : null; + if (request.hasQualifier()) { + scan.addColumn(family, qualifier); + } else { + scan.addFamily(family); + } + int sumResult = 0; + InternalScanner scanner = null; + try { + HRegion region = this.env.getRegion(); + // throw an exception for requests to the last region in the table, in order to test error handling + if (Bytes.equals(region.getEndKey(), HConstants.EMPTY_END_ROW)) { + throw new DoNotRetryIOException("An expected exception"); + } + scanner = region.getScanner(scan); + List curVals = new ArrayList(); + boolean hasMore = false; + do { + curVals.clear(); + hasMore = scanner.next(curVals); + for (Cell kv : curVals) { + if (CellUtil.matchingQualifier(kv, qualifier)) { + sumResult += Bytes.toInt(kv.getValueArray(), kv.getValueOffset()); + } + } + } while (hasMore); + } catch (IOException e) { + ResponseConverter.setControllerException(controller, e); + // Set result to -1 to indicate error. + sumResult = -1; + LOG.info("Setting sum result to -1 to indicate error", e); + } finally { + if (scanner != null) { + try { + scanner.close(); + } catch (IOException e) { + ResponseConverter.setControllerException(controller, e); + sumResult = -1; + LOG.info("Setting sum result to -1 to indicate error", e); + } + } + } + done.run(SumResponse.newBuilder().setSum(sumResult).build()); + } +} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java new file mode 100644 index 0000000..be5f79d --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java @@ -0,0 +1,280 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.coprocessor; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.util.Collections; +import java.util.Map; +import java.util.TreeMap; + +import com.google.protobuf.HBaseZeroCopyByteString; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MediumTests; +import org.apache.hadoop.hbase.TableName; +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.coprocessor.Batch; +import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos; +import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationProtos.SumResponse; +import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos; +import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos; +import org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.ColumnAggregationServiceNullResponse; +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 com.google.protobuf.ServiceException; + +/** + * TestEndpoint: test cases to verify the batch execution of coprocessor Endpoint + */ +@Category(MediumTests.class) +public class TestBatchCoprocessorEndpoint { + private static final Log LOG = LogFactory.getLog(TestBatchCoprocessorEndpoint.class); + + private static final TableName TEST_TABLE = + TableName.valueOf("TestTable"); + private static final byte[] TEST_FAMILY = Bytes.toBytes("TestFamily"); + private static final byte[] TEST_QUALIFIER = Bytes.toBytes("TestQualifier"); + private static byte[] ROW = Bytes.toBytes("testRow"); + + private static final int ROWSIZE = 20; + private static final int rowSeperator1 = 5; + private static final int rowSeperator2 = 12; + private static byte[][] ROWS = makeN(ROW, ROWSIZE); + + private static HBaseTestingUtility util = new HBaseTestingUtility(); + + @BeforeClass + public static void setupBeforeClass() throws Exception { + // 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.class.getName(), + ProtobufCoprocessorService.class.getName(), + ColumnAggregationEndpointWithErrors.class.getName(), + ColumnAggregationEndpointNullResponse.class.getName()); + conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, + ProtobufCoprocessorService.class.getName()); + util.startMiniCluster(2); + 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(TEST_TABLE); + admin.close(); + + HTable table = new HTable(conf, TEST_TABLE); + for (int i = 0; i < ROWSIZE; i++) { + Put put = new Put(ROWS[i]); + put.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes(i)); + table.put(put); + } + table.close(); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + util.shutdownMiniCluster(); + } + + @Test + public void testAggregationNullResponse() throws Throwable { + HTable table = new HTable(util.getConfiguration(), TEST_TABLE); + ColumnAggregationWithNullResponseProtos.SumRequest.Builder builder = + ColumnAggregationWithNullResponseProtos.SumRequest + .newBuilder(); + builder.setFamily(HBaseZeroCopyByteString.wrap(TEST_FAMILY)); + if (TEST_QUALIFIER != null && TEST_QUALIFIER.length > 0) { + builder.setQualifier(HBaseZeroCopyByteString.wrap(TEST_QUALIFIER)); + } + Map results = + table.batchCoprocessorService( + ColumnAggregationServiceNullResponse.getDescriptor().findMethodByName("sum"), + builder.build(), ROWS[0], ROWS[ROWS.length - 1], + ColumnAggregationWithNullResponseProtos.SumResponse.getDefaultInstance()); + + int sumResult = 0; + int expectedResult = 0; + for (Map.Entry e : + results.entrySet()) { + LOG.info("Got value " + e.getValue().getSum() + " for region " + + Bytes.toStringBinary(e.getKey())); + sumResult += e.getValue().getSum(); + } + for (int i = 0; i < rowSeperator2; i++) { + expectedResult += i; + } + assertEquals("Invalid result", expectedResult, sumResult); + table.close(); + } + + private static byte[][] makeN(byte[] base, int n) { + byte[][] ret = new byte[n][]; + for (int i = 0; i < n; i++) { + ret[i] = Bytes.add(base, Bytes.toBytes(String.format("%02d", i))); + } + return ret; + } + + private Map sum(final HTable table, final byte[] family, + final byte[] qualifier, final byte[] start, final byte[] end) throws ServiceException, + Throwable { + ColumnAggregationProtos.SumRequest.Builder builder = ColumnAggregationProtos.SumRequest + .newBuilder(); + builder.setFamily(HBaseZeroCopyByteString.wrap(family)); + if (qualifier != null && qualifier.length > 0) { + builder.setQualifier(HBaseZeroCopyByteString.wrap(qualifier)); + } + return table.batchCoprocessorService( + ColumnAggregationProtos.ColumnAggregationService.getDescriptor().findMethodByName("sum"), + builder.build(), start, end, ColumnAggregationProtos.SumResponse.getDefaultInstance()); + } + + @Test + public void testAggregationWithReturnValue() throws Throwable { + HTable table = new HTable(util.getConfiguration(), TEST_TABLE); + Map results = sum(table, TEST_FAMILY, TEST_QUALIFIER, ROWS[0], + ROWS[ROWS.length - 1]); + int sumResult = 0; + int expectedResult = 0; + for (Map.Entry e : results.entrySet()) { + LOG.info("Got value " + e.getValue().getSum() + " for region " + + Bytes.toStringBinary(e.getKey())); + sumResult += e.getValue().getSum(); + } + for (int i = 0; i < ROWSIZE; i++) { + expectedResult += i; + } + assertEquals("Invalid result", expectedResult, sumResult); + + results.clear(); + + // scan: for region 2 and region 3 + results = sum(table, TEST_FAMILY, TEST_QUALIFIER, ROWS[rowSeperator1], + ROWS[ROWS.length - 1]); + sumResult = 0; + expectedResult = 0; + for (Map.Entry e : results.entrySet()) { + LOG.info("Got value " + e.getValue().getSum() + " for region " + + Bytes.toStringBinary(e.getKey())); + sumResult += e.getValue().getSum(); + } + for (int i = rowSeperator1; i < ROWSIZE; i++) { + expectedResult += i; + } + assertEquals("Invalid result", expectedResult, sumResult); + table.close(); + } + + @Test + public void testAggregation() throws Throwable { + HTable table = new HTable(util.getConfiguration(), TEST_TABLE); + Map results = sum(table, TEST_FAMILY, TEST_QUALIFIER, + ROWS[0], ROWS[ROWS.length - 1]); + int sumResult = 0; + int expectedResult = 0; + for (Map.Entry e : results.entrySet()) { + LOG.info("Got value " + e.getValue().getSum() + " for region " + + Bytes.toStringBinary(e.getKey())); + sumResult += e.getValue().getSum(); + } + for (int i = 0; i < ROWSIZE; i++) { + expectedResult += i; + } + assertEquals("Invalid result", expectedResult, sumResult); + + // scan: for region 2 and region 3 + results = sum(table, TEST_FAMILY, TEST_QUALIFIER, ROWS[rowSeperator1], ROWS[ROWS.length - 1]); + sumResult = 0; + expectedResult = 0; + for (Map.Entry e : results.entrySet()) { + LOG.info("Got value " + e.getValue().getSum() + " for region " + + Bytes.toStringBinary(e.getKey())); + sumResult += e.getValue().getSum(); + } + for (int i = rowSeperator1; i < ROWSIZE; i++) { + expectedResult += i; + } + assertEquals("Invalid result", expectedResult, sumResult); + table.close(); + } + + @Test + public void testAggregationWithErrors() throws Throwable { + HTable table = new HTable(util.getConfiguration(), TEST_TABLE); + final Map results = + Collections.synchronizedMap( + new TreeMap( + Bytes.BYTES_COMPARATOR + )); + ColumnAggregationWithErrorsProtos.SumRequest.Builder builder = + ColumnAggregationWithErrorsProtos.SumRequest + .newBuilder(); + builder.setFamily(HBaseZeroCopyByteString.wrap(TEST_FAMILY)); + if (TEST_QUALIFIER != null && TEST_QUALIFIER.length > 0) { + builder.setQualifier(HBaseZeroCopyByteString.wrap(TEST_QUALIFIER)); + } + + boolean hasError = false; + try { + table.batchCoprocessorService( + ColumnAggregationWithErrorsProtos.ColumnAggregationServiceWithErrors.getDescriptor() + .findMethodByName("sum"), + builder.build(), ROWS[0], ROWS[ROWS.length - 1], + ColumnAggregationWithErrorsProtos.SumResponse.getDefaultInstance(), + new Batch.Callback() { + + @Override + public void update(byte[] region, byte[] row, + ColumnAggregationWithErrorsProtos.SumResponse result) { + results.put(region, result); + } + }); + } catch (Throwable t) { + LOG.info("Exceptions in coprocessor service", t); + hasError = true; + } + + int sumResult = 0; + int expectedResult = 0; + for (Map.Entry e : results.entrySet()) { + LOG.info("Got value " + e.getValue().getSum() + " for region " + + Bytes.toStringBinary(e.getKey())); + sumResult += e.getValue().getSum(); + } + for (int i = 0; i < rowSeperator2; i++) { + expectedResult += i; + } + assertEquals("Invalid result", expectedResult, sumResult); + assertTrue(hasError); + table.close(); + } +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/protobuf/generated/ColumnAggregationWithErrorsProtos.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/protobuf/generated/ColumnAggregationWithErrorsProtos.java new file mode 100644 index 0000000..6768aa2 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/protobuf/generated/ColumnAggregationWithErrorsProtos.java @@ -0,0 +1,1278 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: ColumnAggregationWithErrorsProtocol.proto + +package org.apache.hadoop.hbase.coprocessor.protobuf.generated; + +public final class ColumnAggregationWithErrorsProtos { + private ColumnAggregationWithErrorsProtos() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + } + public interface SumRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required bytes family = 1; + /** + * required bytes family = 1; + */ + boolean hasFamily(); + /** + * required bytes family = 1; + */ + com.google.protobuf.ByteString getFamily(); + + // optional bytes qualifier = 2; + /** + * optional bytes qualifier = 2; + */ + boolean hasQualifier(); + /** + * optional bytes qualifier = 2; + */ + com.google.protobuf.ByteString getQualifier(); + } + /** + * Protobuf type {@code SumRequest} + */ + public static final class SumRequest extends + com.google.protobuf.GeneratedMessage + implements SumRequestOrBuilder { + // Use SumRequest.newBuilder() to construct. + private SumRequest(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private SumRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final SumRequest defaultInstance; + public static SumRequest getDefaultInstance() { + return defaultInstance; + } + + public SumRequest getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private SumRequest( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + family_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + qualifier_ = input.readBytes(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.internal_static_SumRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.internal_static_SumRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest.class, org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public SumRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new SumRequest(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required bytes family = 1; + public static final int FAMILY_FIELD_NUMBER = 1; + private com.google.protobuf.ByteString family_; + /** + * required bytes family = 1; + */ + public boolean hasFamily() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required bytes family = 1; + */ + public com.google.protobuf.ByteString getFamily() { + return family_; + } + + // optional bytes qualifier = 2; + public static final int QUALIFIER_FIELD_NUMBER = 2; + private com.google.protobuf.ByteString qualifier_; + /** + * optional bytes qualifier = 2; + */ + public boolean hasQualifier() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional bytes qualifier = 2; + */ + public com.google.protobuf.ByteString getQualifier() { + return qualifier_; + } + + private void initFields() { + family_ = com.google.protobuf.ByteString.EMPTY; + qualifier_ = com.google.protobuf.ByteString.EMPTY; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasFamily()) { + 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, family_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, qualifier_); + } + 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, family_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, qualifier_); + } + 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.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest other = (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest) obj; + + boolean result = true; + result = result && (hasFamily() == other.hasFamily()); + if (hasFamily()) { + result = result && getFamily() + .equals(other.getFamily()); + } + result = result && (hasQualifier() == other.hasQualifier()); + if (hasQualifier()) { + result = result && getQualifier() + .equals(other.getQualifier()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasFamily()) { + hash = (37 * hash) + FAMILY_FIELD_NUMBER; + hash = (53 * hash) + getFamily().hashCode(); + } + if (hasQualifier()) { + hash = (37 * hash) + QUALIFIER_FIELD_NUMBER; + hash = (53 * hash) + getQualifier().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest 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; + } + /** + * Protobuf type {@code SumRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.internal_static_SumRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.internal_static_SumRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest.class, org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.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(); + family_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + qualifier_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.internal_static_SumRequest_descriptor; + } + + public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest build() { + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest buildPartial() { + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest result = new org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.family_ = family_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.qualifier_ = qualifier_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest) { + return mergeFrom((org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest other) { + if (other == org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest.getDefaultInstance()) return this; + if (other.hasFamily()) { + setFamily(other.getFamily()); + } + if (other.hasQualifier()) { + setQualifier(other.getQualifier()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasFamily()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required bytes family = 1; + private com.google.protobuf.ByteString family_ = com.google.protobuf.ByteString.EMPTY; + /** + * required bytes family = 1; + */ + public boolean hasFamily() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required bytes family = 1; + */ + public com.google.protobuf.ByteString getFamily() { + return family_; + } + /** + * required bytes family = 1; + */ + public Builder setFamily(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + family_ = value; + onChanged(); + return this; + } + /** + * required bytes family = 1; + */ + public Builder clearFamily() { + bitField0_ = (bitField0_ & ~0x00000001); + family_ = getDefaultInstance().getFamily(); + onChanged(); + return this; + } + + // optional bytes qualifier = 2; + private com.google.protobuf.ByteString qualifier_ = com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes qualifier = 2; + */ + public boolean hasQualifier() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional bytes qualifier = 2; + */ + public com.google.protobuf.ByteString getQualifier() { + return qualifier_; + } + /** + * optional bytes qualifier = 2; + */ + public Builder setQualifier(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + qualifier_ = value; + onChanged(); + return this; + } + /** + * optional bytes qualifier = 2; + */ + public Builder clearQualifier() { + bitField0_ = (bitField0_ & ~0x00000002); + qualifier_ = getDefaultInstance().getQualifier(); + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:SumRequest) + } + + static { + defaultInstance = new SumRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:SumRequest) + } + + public interface SumResponseOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required int64 sum = 1; + /** + * required int64 sum = 1; + */ + boolean hasSum(); + /** + * required int64 sum = 1; + */ + long getSum(); + } + /** + * Protobuf type {@code SumResponse} + */ + public static final class SumResponse extends + com.google.protobuf.GeneratedMessage + implements SumResponseOrBuilder { + // Use SumResponse.newBuilder() to construct. + private SumResponse(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private SumResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final SumResponse defaultInstance; + public static SumResponse getDefaultInstance() { + return defaultInstance; + } + + public SumResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private SumResponse( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + sum_ = input.readInt64(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.internal_static_SumResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.internal_static_SumResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse.class, org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public SumResponse parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new SumResponse(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required int64 sum = 1; + public static final int SUM_FIELD_NUMBER = 1; + private long sum_; + /** + * required int64 sum = 1; + */ + public boolean hasSum() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required int64 sum = 1; + */ + public long getSum() { + return sum_; + } + + private void initFields() { + sum_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasSum()) { + 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, sum_); + } + 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, sum_); + } + 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.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse other = (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse) obj; + + boolean result = true; + result = result && (hasSum() == other.hasSum()); + if (hasSum()) { + result = result && (getSum() + == other.getSum()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasSum()) { + hash = (37 * hash) + SUM_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getSum()); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse 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; + } + /** + * Protobuf type {@code SumResponse} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.internal_static_SumResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.internal_static_SumResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse.class, org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.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(); + sum_ = 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.protobuf.generated.ColumnAggregationWithErrorsProtos.internal_static_SumResponse_descriptor; + } + + public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse build() { + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse buildPartial() { + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse result = new org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.sum_ = sum_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse) { + return mergeFrom((org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse other) { + if (other == org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse.getDefaultInstance()) return this; + if (other.hasSum()) { + setSum(other.getSum()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasSum()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required int64 sum = 1; + private long sum_ ; + /** + * required int64 sum = 1; + */ + public boolean hasSum() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required int64 sum = 1; + */ + public long getSum() { + return sum_; + } + /** + * required int64 sum = 1; + */ + public Builder setSum(long value) { + bitField0_ |= 0x00000001; + sum_ = value; + onChanged(); + return this; + } + /** + * required int64 sum = 1; + */ + public Builder clearSum() { + bitField0_ = (bitField0_ & ~0x00000001); + sum_ = 0L; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:SumResponse) + } + + static { + defaultInstance = new SumResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:SumResponse) + } + + /** + * Protobuf service {@code ColumnAggregationServiceWithErrors} + */ + public static abstract class ColumnAggregationServiceWithErrors + implements com.google.protobuf.Service { + protected ColumnAggregationServiceWithErrors() {} + + public interface Interface { + /** + * rpc sum(.SumRequest) returns (.SumResponse); + */ + public abstract void sum( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest request, + com.google.protobuf.RpcCallback done); + + } + + public static com.google.protobuf.Service newReflectiveService( + final Interface impl) { + return new ColumnAggregationServiceWithErrors() { + @java.lang.Override + public void sum( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest request, + com.google.protobuf.RpcCallback done) { + impl.sum(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.sum(controller, (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest)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.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest.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.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + }; + } + + /** + * rpc sum(.SumRequest) returns (.SumResponse); + */ + public abstract void sum( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest request, + com.google.protobuf.RpcCallback done); + + public static final + com.google.protobuf.Descriptors.ServiceDescriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.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.sum(controller, (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest)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.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest.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.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse.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.protobuf.generated.ColumnAggregationWithErrorsProtos.ColumnAggregationServiceWithErrors 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 sum( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(0), + controller, + request, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse.class, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse.getDefaultInstance())); + } + } + + public static BlockingInterface newBlockingStub( + com.google.protobuf.BlockingRpcChannel channel) { + return new BlockingStub(channel); + } + + public interface BlockingInterface { + public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse sum( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest 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.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse sum( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(0), + controller, + request, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithErrorsProtos.SumResponse.getDefaultInstance()); + } + + } + + // @@protoc_insertion_point(class_scope:ColumnAggregationServiceWithErrors) + } + + private static com.google.protobuf.Descriptors.Descriptor + internal_static_SumRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_SumRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_SumResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_SumResponse_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)ColumnAggregationWithErrorsProtocol.pr" + + "oto\"/\n\nSumRequest\022\016\n\006family\030\001 \002(\014\022\021\n\tqua" + + "lifier\030\002 \001(\014\"\032\n\013SumResponse\022\013\n\003sum\030\001 \002(\003" + + "2F\n\"ColumnAggregationServiceWithErrors\022 " + + "\n\003sum\022\013.SumRequest\032\014.SumResponseBa\n6org." + + "apache.hadoop.hbase.coprocessor.protobuf" + + ".generatedB!ColumnAggregationWithErrorsP" + + "rotos\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_SumRequest_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_SumRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_SumRequest_descriptor, + new java.lang.String[] { "Family", "Qualifier", }); + internal_static_SumResponse_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_SumResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_SumResponse_descriptor, + new java.lang.String[] { "Sum", }); + 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/test/java/org/apache/hadoop/hbase/coprocessor/protobuf/generated/ColumnAggregationWithNullResponseProtos.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/protobuf/generated/ColumnAggregationWithNullResponseProtos.java new file mode 100644 index 0000000..c0b1917 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/protobuf/generated/ColumnAggregationWithNullResponseProtos.java @@ -0,0 +1,1270 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: ColumnAggregationNullResponseProtocol.proto + +package org.apache.hadoop.hbase.coprocessor.protobuf.generated; + +public final class ColumnAggregationWithNullResponseProtos { + private ColumnAggregationWithNullResponseProtos() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + } + public interface SumRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required bytes family = 1; + /** + * required bytes family = 1; + */ + boolean hasFamily(); + /** + * required bytes family = 1; + */ + com.google.protobuf.ByteString getFamily(); + + // optional bytes qualifier = 2; + /** + * optional bytes qualifier = 2; + */ + boolean hasQualifier(); + /** + * optional bytes qualifier = 2; + */ + com.google.protobuf.ByteString getQualifier(); + } + /** + * Protobuf type {@code SumRequest} + */ + public static final class SumRequest extends + com.google.protobuf.GeneratedMessage + implements SumRequestOrBuilder { + // Use SumRequest.newBuilder() to construct. + private SumRequest(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private SumRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final SumRequest defaultInstance; + public static SumRequest getDefaultInstance() { + return defaultInstance; + } + + public SumRequest getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private SumRequest( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + family_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + qualifier_ = input.readBytes(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.internal_static_SumRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.internal_static_SumRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest.class, org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public SumRequest parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new SumRequest(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // required bytes family = 1; + public static final int FAMILY_FIELD_NUMBER = 1; + private com.google.protobuf.ByteString family_; + /** + * required bytes family = 1; + */ + public boolean hasFamily() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required bytes family = 1; + */ + public com.google.protobuf.ByteString getFamily() { + return family_; + } + + // optional bytes qualifier = 2; + public static final int QUALIFIER_FIELD_NUMBER = 2; + private com.google.protobuf.ByteString qualifier_; + /** + * optional bytes qualifier = 2; + */ + public boolean hasQualifier() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional bytes qualifier = 2; + */ + public com.google.protobuf.ByteString getQualifier() { + return qualifier_; + } + + private void initFields() { + family_ = com.google.protobuf.ByteString.EMPTY; + qualifier_ = com.google.protobuf.ByteString.EMPTY; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasFamily()) { + 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, family_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, qualifier_); + } + 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, family_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, qualifier_); + } + 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.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest other = (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest) obj; + + boolean result = true; + result = result && (hasFamily() == other.hasFamily()); + if (hasFamily()) { + result = result && getFamily() + .equals(other.getFamily()); + } + result = result && (hasQualifier() == other.hasQualifier()); + if (hasQualifier()) { + result = result && getQualifier() + .equals(other.getQualifier()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasFamily()) { + hash = (37 * hash) + FAMILY_FIELD_NUMBER; + hash = (53 * hash) + getFamily().hashCode(); + } + if (hasQualifier()) { + hash = (37 * hash) + QUALIFIER_FIELD_NUMBER; + hash = (53 * hash) + getQualifier().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest 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; + } + /** + * Protobuf type {@code SumRequest} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.internal_static_SumRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.internal_static_SumRequest_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest.class, org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.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(); + family_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000001); + qualifier_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.internal_static_SumRequest_descriptor; + } + + public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest build() { + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest buildPartial() { + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest result = new org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.family_ = family_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.qualifier_ = qualifier_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest) { + return mergeFrom((org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest other) { + if (other == org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest.getDefaultInstance()) return this; + if (other.hasFamily()) { + setFamily(other.getFamily()); + } + if (other.hasQualifier()) { + setQualifier(other.getQualifier()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasFamily()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // required bytes family = 1; + private com.google.protobuf.ByteString family_ = com.google.protobuf.ByteString.EMPTY; + /** + * required bytes family = 1; + */ + public boolean hasFamily() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * required bytes family = 1; + */ + public com.google.protobuf.ByteString getFamily() { + return family_; + } + /** + * required bytes family = 1; + */ + public Builder setFamily(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + family_ = value; + onChanged(); + return this; + } + /** + * required bytes family = 1; + */ + public Builder clearFamily() { + bitField0_ = (bitField0_ & ~0x00000001); + family_ = getDefaultInstance().getFamily(); + onChanged(); + return this; + } + + // optional bytes qualifier = 2; + private com.google.protobuf.ByteString qualifier_ = com.google.protobuf.ByteString.EMPTY; + /** + * optional bytes qualifier = 2; + */ + public boolean hasQualifier() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * optional bytes qualifier = 2; + */ + public com.google.protobuf.ByteString getQualifier() { + return qualifier_; + } + /** + * optional bytes qualifier = 2; + */ + public Builder setQualifier(com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + qualifier_ = value; + onChanged(); + return this; + } + /** + * optional bytes qualifier = 2; + */ + public Builder clearQualifier() { + bitField0_ = (bitField0_ & ~0x00000002); + qualifier_ = getDefaultInstance().getQualifier(); + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:SumRequest) + } + + static { + defaultInstance = new SumRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:SumRequest) + } + + public interface SumResponseOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // optional int64 sum = 1; + /** + * optional int64 sum = 1; + */ + boolean hasSum(); + /** + * optional int64 sum = 1; + */ + long getSum(); + } + /** + * Protobuf type {@code SumResponse} + */ + public static final class SumResponse extends + com.google.protobuf.GeneratedMessage + implements SumResponseOrBuilder { + // Use SumResponse.newBuilder() to construct. + private SumResponse(com.google.protobuf.GeneratedMessage.Builder builder) { + super(builder); + this.unknownFields = builder.getUnknownFields(); + } + private SumResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); } + + private static final SumResponse defaultInstance; + public static SumResponse getDefaultInstance() { + return defaultInstance; + } + + public SumResponse getDefaultInstanceForType() { + return defaultInstance; + } + + private final com.google.protobuf.UnknownFieldSet unknownFields; + @java.lang.Override + public final com.google.protobuf.UnknownFieldSet + getUnknownFields() { + return this.unknownFields; + } + private SumResponse( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + initFields(); + int mutable_bitField0_ = 0; + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder(); + try { + boolean done = false; + while (!done) { + int tag = input.readTag(); + switch (tag) { + case 0: + done = true; + break; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + done = true; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + sum_ = input.readInt64(); + break; + } + } + } + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw e.setUnfinishedMessage(this); + } catch (java.io.IOException e) { + throw new com.google.protobuf.InvalidProtocolBufferException( + e.getMessage()).setUnfinishedMessage(this); + } finally { + this.unknownFields = unknownFields.build(); + makeExtensionsImmutable(); + } + } + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.internal_static_SumResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.internal_static_SumResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse.class, org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse.Builder.class); + } + + public static com.google.protobuf.Parser PARSER = + new com.google.protobuf.AbstractParser() { + public SumResponse parsePartialFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return new SumResponse(input, extensionRegistry); + } + }; + + @java.lang.Override + public com.google.protobuf.Parser getParserForType() { + return PARSER; + } + + private int bitField0_; + // optional int64 sum = 1; + public static final int SUM_FIELD_NUMBER = 1; + private long sum_; + /** + * optional int64 sum = 1; + */ + public boolean hasSum() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional int64 sum = 1; + */ + public long getSum() { + return sum_; + } + + private void initFields() { + sum_ = 0L; + } + 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(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeInt64(1, sum_); + } + 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, sum_); + } + 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.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse other = (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse) obj; + + boolean result = true; + result = result && (hasSum() == other.hasSum()); + if (hasSum()) { + result = result && (getSum() + == other.getSum()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + private int memoizedHashCode = 0; + @java.lang.Override + public int hashCode() { + if (memoizedHashCode != 0) { + return memoizedHashCode; + } + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasSum()) { + hash = (37 * hash) + SUM_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getSum()); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + memoizedHashCode = hash; + return hash; + } + + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return PARSER.parseFrom(data, extensionRegistry); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseDelimitedFrom(input, extensionRegistry); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return PARSER.parseFrom(input); + } + public static org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return PARSER.parseFrom(input, extensionRegistry); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse 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; + } + /** + * Protobuf type {@code SumResponse} + */ + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.internal_static_SumResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.internal_static_SumResponse_fieldAccessorTable + .ensureFieldAccessorsInitialized( + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse.class, org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse.Builder.class); + } + + // Construct using org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder( + com.google.protobuf.GeneratedMessage.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(); + sum_ = 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.protobuf.generated.ColumnAggregationWithNullResponseProtos.internal_static_SumResponse_descriptor; + } + + public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse build() { + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse buildPartial() { + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse result = new org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.sum_ = sum_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse) { + return mergeFrom((org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse other) { + if (other == org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse.getDefaultInstance()) return this; + if (other.hasSum()) { + setSum(other.getSum()); + } + 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 { + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse parsedMessage = null; + try { + parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + parsedMessage = (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse) e.getUnfinishedMessage(); + throw e; + } finally { + if (parsedMessage != null) { + mergeFrom(parsedMessage); + } + } + return this; + } + private int bitField0_; + + // optional int64 sum = 1; + private long sum_ ; + /** + * optional int64 sum = 1; + */ + public boolean hasSum() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + /** + * optional int64 sum = 1; + */ + public long getSum() { + return sum_; + } + /** + * optional int64 sum = 1; + */ + public Builder setSum(long value) { + bitField0_ |= 0x00000001; + sum_ = value; + onChanged(); + return this; + } + /** + * optional int64 sum = 1; + */ + public Builder clearSum() { + bitField0_ = (bitField0_ & ~0x00000001); + sum_ = 0L; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:SumResponse) + } + + static { + defaultInstance = new SumResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:SumResponse) + } + + /** + * Protobuf service {@code ColumnAggregationServiceNullResponse} + */ + public static abstract class ColumnAggregationServiceNullResponse + implements com.google.protobuf.Service { + protected ColumnAggregationServiceNullResponse() {} + + public interface Interface { + /** + * rpc sum(.SumRequest) returns (.SumResponse); + */ + public abstract void sum( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest request, + com.google.protobuf.RpcCallback done); + + } + + public static com.google.protobuf.Service newReflectiveService( + final Interface impl) { + return new ColumnAggregationServiceNullResponse() { + @java.lang.Override + public void sum( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest request, + com.google.protobuf.RpcCallback done) { + impl.sum(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.sum(controller, (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest)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.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest.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.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + }; + } + + /** + * rpc sum(.SumRequest) returns (.SumResponse); + */ + public abstract void sum( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest request, + com.google.protobuf.RpcCallback done); + + public static final + com.google.protobuf.Descriptors.ServiceDescriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.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.sum(controller, (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest)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.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest.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.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse.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.protobuf.generated.ColumnAggregationWithNullResponseProtos.ColumnAggregationServiceNullResponse 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 sum( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(0), + controller, + request, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse.class, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse.getDefaultInstance())); + } + } + + public static BlockingInterface newBlockingStub( + com.google.protobuf.BlockingRpcChannel channel) { + return new BlockingStub(channel); + } + + public interface BlockingInterface { + public org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse sum( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest 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.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse sum( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(0), + controller, + request, + org.apache.hadoop.hbase.coprocessor.protobuf.generated.ColumnAggregationWithNullResponseProtos.SumResponse.getDefaultInstance()); + } + + } + + // @@protoc_insertion_point(class_scope:ColumnAggregationServiceNullResponse) + } + + private static com.google.protobuf.Descriptors.Descriptor + internal_static_SumRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_SumRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_SumResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_SumResponse_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+ColumnAggregationNullResponseProtocol." + + "proto\"/\n\nSumRequest\022\016\n\006family\030\001 \002(\014\022\021\n\tq" + + "ualifier\030\002 \001(\014\"\032\n\013SumResponse\022\013\n\003sum\030\001 \001" + + "(\0032H\n$ColumnAggregationServiceNullRespon" + + "se\022 \n\003sum\022\013.SumRequest\032\014.SumResponseBg\n6" + + "org.apache.hadoop.hbase.coprocessor.prot" + + "obuf.generatedB\'ColumnAggregationWithNul" + + "lResponseProtos\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_SumRequest_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_SumRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_SumRequest_descriptor, + new java.lang.String[] { "Family", "Qualifier", }); + internal_static_SumResponse_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_SumResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_SumResponse_descriptor, + new java.lang.String[] { "Sum", }); + 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/test/protobuf/ColumnAggregationNullResponseProtocol.proto b/hbase-server/src/test/protobuf/ColumnAggregationNullResponseProtocol.proto new file mode 100644 index 0000000..39bb05f --- /dev/null +++ b/hbase-server/src/test/protobuf/ColumnAggregationNullResponseProtocol.proto @@ -0,0 +1,35 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +// Coprocessor test +option java_package = "org.apache.hadoop.hbase.coprocessor.protobuf.generated"; +option java_outer_classname = "ColumnAggregationWithNullResponseProtos"; +option java_generic_services = true; +option java_generate_equals_and_hash = true; + +message SumRequest { + required bytes family = 1; + optional bytes qualifier = 2; +} + +message SumResponse { + optional int64 sum = 1; +} + +service ColumnAggregationServiceNullResponse { + rpc sum(SumRequest) returns(SumResponse); +} diff --git a/hbase-server/src/test/protobuf/ColumnAggregationWithErrorsProtocol.proto b/hbase-server/src/test/protobuf/ColumnAggregationWithErrorsProtocol.proto new file mode 100644 index 0000000..c195c33 --- /dev/null +++ b/hbase-server/src/test/protobuf/ColumnAggregationWithErrorsProtocol.proto @@ -0,0 +1,35 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +// Coprocessor test +option java_package = "org.apache.hadoop.hbase.coprocessor.protobuf.generated"; +option java_outer_classname = "ColumnAggregationWithErrorsProtos"; +option java_generic_services = true; +option java_generate_equals_and_hash = true; + +message SumRequest { + required bytes family = 1; + optional bytes qualifier = 2; +} + +message SumResponse { + required int64 sum = 1; +} + +service ColumnAggregationServiceWithErrors { + rpc sum(SumRequest) returns(SumResponse); +}