You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2014/03/12 16:59:01 UTC
svn commit: r1576793 [1/3] - in /hbase/branches/0.98:
hbase-client/src/main/java/org/apache/hadoop/hbase/client/
hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/
hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/
hbase...
Author: apurtell
Date: Wed Mar 12 15:59:00 2014
New Revision: 1576793
URL: http://svn.apache.org/r1576793
Log:
HBASE-10169 Batch coprocessor (Jingcheng Du and Gary Helmling)
Added:
hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorServiceExec.java
hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointNullResponse.java
hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpointWithErrors.java
hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestBatchCoprocessorEndpoint.java
hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/protobuf/generated/ColumnAggregationWithErrorsProtos.java
hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/protobuf/generated/ColumnAggregationWithNullResponseProtos.java
hbase/branches/0.98/hbase-server/src/test/protobuf/ColumnAggregationNullResponseProtocol.proto
hbase/branches/0.98/hbase-server/src/test/protobuf/ColumnAggregationWithErrorsProtocol.proto
Modified:
hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java
hbase/branches/0.98/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
hbase/branches/0.98/hbase-protocol/src/main/protobuf/Client.proto
hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java
hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/ColumnAggregationEndpoint.java
Modified: hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java?rev=1576793&r1=1576792&r2=1576793&view=diff
==============================================================================
--- hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java Wed Mar 12 15:59:00 2014
@@ -36,12 +36,14 @@ import java.util.concurrent.SynchronousQ
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,12 +54,14 @@ import org.apache.hadoop.hbase.KeyValueU
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.ipc.CoprocessorRpcChannel;
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;
@@ -67,6 +71,8 @@ import org.apache.hadoop.hbase.util.Byte
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;
@@ -1374,8 +1380,7 @@ public class HTable implements HTableInt
public static void setRegionCachePrefetch(
final TableName tableName,
final boolean enable) throws IOException {
- HConnectionManager.execute(new HConnectable<Void>(HBaseConfiguration
- .create()) {
+ HConnectionManager.execute(new HConnectable<Void>(HBaseConfiguration.create()) {
@Override
public Void connect(HConnection connection) throws IOException {
connection.setRegionCachePrefetch(tableName, enable);
@@ -1572,4 +1577,123 @@ public class HTable implements HTableInt
t.close();
}
}
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public <R extends Message> Map<byte[], R> batchCoprocessorService(
+ Descriptors.MethodDescriptor methodDescriptor, Message request,
+ byte[] startKey, byte[] endKey, R responsePrototype) throws ServiceException, Throwable {
+ final Map<byte[], R> results = Collections.synchronizedMap(new TreeMap<byte[], R>(
+ Bytes.BYTES_COMPARATOR));
+ batchCoprocessorService(methodDescriptor, request, startKey, endKey, responsePrototype,
+ new Callback<R>() {
+
+ @Override
+ public void update(byte[] region, byte[] row, R result) {
+ if (region != null) {
+ results.put(region, result);
+ }
+ }
+ });
+ return results;
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public <R extends Message> void batchCoprocessorService(
+ final Descriptors.MethodDescriptor methodDescriptor, final Message request,
+ byte[] startKey, byte[] endKey, final R responsePrototype, final Callback<R> callback)
+ throws ServiceException, Throwable {
+
+ // get regions covered by the row range
+ Pair<List<byte[]>, List<HRegionLocation>> keysAndRegions =
+ getKeysAndRegionsInRange(startKey, endKey, true);
+ List<byte[]> keys = keysAndRegions.getFirst();
+ List<HRegionLocation> regions = keysAndRegions.getSecond();
+
+ // check if we have any calls to make
+ if (keys.isEmpty()) {
+ LOG.info("No regions were selected by key range start=" + Bytes.toStringBinary(startKey) +
+ ", end=" + Bytes.toStringBinary(endKey));
+ return;
+ }
+
+ List<RegionCoprocessorServiceExec> execs = new ArrayList<RegionCoprocessorServiceExec>();
+ final Map<byte[], RegionCoprocessorServiceExec> execsByRow =
+ new TreeMap<byte[], RegionCoprocessorServiceExec>(Bytes.BYTES_COMPARATOR);
+ for (int i = 0; i < keys.size(); i++) {
+ final byte[] rowKey = keys.get(i);
+ final byte[] region = regions.get(i).getRegionInfo().getRegionName();
+ RegionCoprocessorServiceExec exec =
+ new RegionCoprocessorServiceExec(region, rowKey, methodDescriptor, request);
+ execs.add(exec);
+ execsByRow.put(rowKey, exec);
+ }
+
+ // tracking for any possible deserialization errors on success callback
+ // TODO: it would be better to be able to reuse AsyncProcess.BatchErrors here
+ final List<Throwable> callbackErrorExceptions = new ArrayList<Throwable>();
+ final List<Row> callbackErrorActions = new ArrayList<Row>();
+ final List<String> callbackErrorServers = new ArrayList<String>();
+
+ AsyncProcess<ClientProtos.CoprocessorServiceResult> asyncProcess =
+ new AsyncProcess<ClientProtos.CoprocessorServiceResult>(connection, tableName, pool,
+ new AsyncProcess.AsyncProcessCallback<ClientProtos.CoprocessorServiceResult>() {
+ @SuppressWarnings("unchecked")
+ @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, byte[] region, 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, byte[] region,
+ 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);
+ }
+ }
}
Modified: hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java?rev=1576793&r1=1576792&r2=1576793&view=diff
==============================================================================
--- hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java (original)
+++ hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTableInterface.java Wed Mar 12 15:59:00 2014
@@ -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;
@@ -585,4 +587,69 @@ public interface HTableInterface extends
* @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 <R>
+ * the response type for the coprocessor Service method
+ * @throws ServiceException
+ * @throws Throwable
+ * @return a map of result values keyed by region name
+ */
+ @InterfaceAudience.Private
+ <R extends Message> Map<byte[], R> 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.
+ *
+ * <p>
+ * 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.
+ * </p>
+ *
+ * @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 <R>
+ * the response type for the coprocessor Service method
+ * @throws ServiceException
+ * @throws Throwable
+ */
+ @InterfaceAudience.Private
+ <R extends Message> void batchCoprocessorService(Descriptors.MethodDescriptor methodDescriptor,
+ Message request, byte[] startKey, byte[] endKey, R responsePrototype,
+ Batch.Callback<R> callback) throws ServiceException, Throwable;
}
Modified: hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java?rev=1576793&r1=1576792&r2=1576793&view=diff
==============================================================================
--- hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java (original)
+++ hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTablePool.java Wed Mar 12 15:59:00 2014
@@ -36,6 +36,8 @@ import org.apache.hadoop.hbase.util.Byte
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;
@@ -622,5 +624,23 @@ public class HTablePool implements Close
byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
return table.incrementColumnValue(row, family, qualifier, amount, writeToWAL);
}
+
+ @Override
+ public <R extends Message> Map<byte[], R> 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 <R extends Message> void batchCoprocessorService(
+ Descriptors.MethodDescriptor method, Message request,
+ byte[] startKey, byte[] endKey, R responsePrototype, Callback<R> callback)
+ throws ServiceException, Throwable {
+ checkState();
+ table.batchCoprocessorService(method, request, startKey, endKey, responsePrototype, callback);
+ }
}
}
Added: hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorServiceExec.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorServiceExec.java?rev=1576793&view=auto
==============================================================================
--- hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorServiceExec.java (added)
+++ hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionCoprocessorServiceExec.java Wed Mar 12 15:59:00 2014
@@ -0,0 +1,108 @@
+/**
+ *
+ * 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.
+ *
+ * <p><b>Note:</b> This class should not be instantiated directly. Use either
+ * {@link HTable#batchCoprocessorService(MethodDescriptor, Message, byte[], byte[],
+ * Message, Batch.Callback)}
+ * or {@link HTable#batchCoprocessorService(MethodDescriptor, Message, byte[], byte[], Message)}
+ * instead.</p>
+ */
+@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;
+ }
+}
Modified: hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java?rev=1576793&r1=1576792&r2=1576793&view=diff
==============================================================================
--- hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (original)
+++ hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java Wed Mar 12 15:59:00 2014
@@ -38,6 +38,7 @@ import org.apache.hadoop.hbase.client.Ge
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 {
Modified: hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java?rev=1576793&r1=1576792&r2=1576793&view=diff
==============================================================================
--- hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java (original)
+++ hbase/branches/0.98/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ResponseConverter.java Wed Mar 12 15:59:00 2014
@@ -122,6 +122,8 @@ public final class ResponseConverter {
} else if (roe.hasResult()) {
results.add(regionName, new Pair<Integer, Object>(roe.getIndex(),
ProtobufUtil.toResult(roe.getResult(), cells)));
+ } else if (roe.hasServiceResult()) {
+ results.add(regionName, roe.getIndex(), roe.getServiceResult());
} else {
// no result & no exception. Unexpected.
throw new IllegalStateException("No result & no exception roe=" + roe +
Modified: hbase/branches/0.98/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.98/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java?rev=1576793&r1=1576792&r2=1576793&view=diff
==============================================================================
--- hbase/branches/0.98/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java (original)
+++ hbase/branches/0.98/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java Wed Mar 12 15:59:00 2014
@@ -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;
+ /**
+ * <code>optional .NameBytesPair value = 1;</code>
+ */
+ boolean hasValue();
+ /**
+ * <code>optional .NameBytesPair value = 1;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getValue();
+ /**
+ * <code>optional .NameBytesPair value = 1;</code>
+ */
+ 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<CoprocessorServiceResult> PARSER =
+ new com.google.protobuf.AbstractParser<CoprocessorServiceResult>() {
+ 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<CoprocessorServiceResult> 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_;
+ /**
+ * <code>optional .NameBytesPair value = 1;</code>
+ */
+ public boolean hasValue() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional .NameBytesPair value = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getValue() {
+ return value_;
+ }
+ /**
+ * <code>optional .NameBytesPair value = 1;</code>
+ */
+ 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<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_;
+ /**
+ * <code>optional .NameBytesPair value = 1;</code>
+ */
+ public boolean hasValue() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional .NameBytesPair value = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair getValue() {
+ if (valueBuilder_ == null) {
+ return value_;
+ } else {
+ return valueBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>optional .NameBytesPair value = 1;</code>
+ */
+ 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;
+ }
+ /**
+ * <code>optional .NameBytesPair value = 1;</code>
+ */
+ 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;
+ }
+ /**
+ * <code>optional .NameBytesPair value = 1;</code>
+ */
+ 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;
+ }
+ /**
+ * <code>optional .NameBytesPair value = 1;</code>
+ */
+ 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;
+ }
+ /**
+ * <code>optional .NameBytesPair value = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPair.Builder getValueBuilder() {
+ bitField0_ |= 0x00000001;
+ onChanged();
+ return getValueFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>optional .NameBytesPair value = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getValueOrBuilder() {
+ if (valueBuilder_ != null) {
+ return valueBuilder_.getMessageOrBuilder();
+ } else {
+ return value_;
+ }
+ }
+ /**
+ * <code>optional .NameBytesPair value = 1;</code>
+ */
+ 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 {
* <code>optional .Get get = 3;</code>
*/
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.GetOrBuilder getGetOrBuilder();
+
+ // optional .CoprocessorServiceCall service_call = 4;
+ /**
+ * <code>optional .CoprocessorServiceCall service_call = 4;</code>
+ */
+ boolean hasServiceCall();
+ /**
+ * <code>optional .CoprocessorServiceCall service_call = 4;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall getServiceCall();
+ /**
+ * <code>optional .CoprocessorServiceCall service_call = 4;</code>
+ */
+ 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_;
+ /**
+ * <code>optional .CoprocessorServiceCall service_call = 4;</code>
+ */
+ public boolean hasServiceCall() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional .CoprocessorServiceCall service_call = 4;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall getServiceCall() {
+ return serviceCall_;
+ }
+ /**
+ * <code>optional .CoprocessorServiceCall service_call = 4;</code>
+ */
+ 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;
}
@@ -23418,6 +24054,14 @@ public final class ClientProtos {
} else {
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();
return result;
@@ -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_;
+ /**
+ * <code>optional .CoprocessorServiceCall service_call = 4;</code>
+ */
+ public boolean hasServiceCall() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional .CoprocessorServiceCall service_call = 4;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall getServiceCall() {
+ if (serviceCallBuilder_ == null) {
+ return serviceCall_;
+ } else {
+ return serviceCallBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>optional .CoprocessorServiceCall service_call = 4;</code>
+ */
+ 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;
+ }
+ /**
+ * <code>optional .CoprocessorServiceCall service_call = 4;</code>
+ */
+ 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;
+ }
+ /**
+ * <code>optional .CoprocessorServiceCall service_call = 4;</code>
+ */
+ 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;
+ }
+ /**
+ * <code>optional .CoprocessorServiceCall service_call = 4;</code>
+ */
+ 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;
+ }
+ /**
+ * <code>optional .CoprocessorServiceCall service_call = 4;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall.Builder getServiceCallBuilder() {
+ bitField0_ |= 0x00000008;
+ onChanged();
+ return getServiceCallFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>optional .CoprocessorServiceCall service_call = 4;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCallOrBuilder getServiceCallOrBuilder() {
+ if (serviceCallBuilder_ != null) {
+ return serviceCallBuilder_.getMessageOrBuilder();
+ } else {
+ return serviceCall_;
+ }
+ }
+ /**
+ * <code>optional .CoprocessorServiceCall service_call = 4;</code>
+ */
+ 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 {
* <code>optional .NameBytesPair exception = 3;</code>
*/
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameBytesPairOrBuilder getExceptionOrBuilder();
+
+ // optional .CoprocessorServiceResult service_result = 4;
+ /**
+ * <code>optional .CoprocessorServiceResult service_result = 4;</code>
+ *
+ * <pre>
+ * result if this was a coprocessor service call
+ * </pre>
+ */
+ boolean hasServiceResult();
+ /**
+ * <code>optional .CoprocessorServiceResult service_result = 4;</code>
+ *
+ * <pre>
+ * result if this was a coprocessor service call
+ * </pre>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult getServiceResult();
+ /**
+ * <code>optional .CoprocessorServiceResult service_result = 4;</code>
+ *
+ * <pre>
+ * result if this was a coprocessor service call
+ * </pre>
+ */
+ 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_;
+ /**
+ * <code>optional .CoprocessorServiceResult service_result = 4;</code>
+ *
+ * <pre>
+ * result if this was a coprocessor service call
+ * </pre>
+ */
+ public boolean hasServiceResult() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional .CoprocessorServiceResult service_result = 4;</code>
+ *
+ * <pre>
+ * result if this was a coprocessor service call
+ * </pre>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult getServiceResult() {
+ return serviceResult_;
+ }
+ /**
+ * <code>optional .CoprocessorServiceResult service_result = 4;</code>
+ *
+ * <pre>
+ * result if this was a coprocessor service call
+ * </pre>
+ */
+ 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_;
+ /**
+ * <code>optional .CoprocessorServiceResult service_result = 4;</code>
+ *
+ * <pre>
+ * result if this was a coprocessor service call
+ * </pre>
+ */
+ public boolean hasServiceResult() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional .CoprocessorServiceResult service_result = 4;</code>
+ *
+ * <pre>
+ * result if this was a coprocessor service call
+ * </pre>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult getServiceResult() {
+ if (serviceResultBuilder_ == null) {
+ return serviceResult_;
+ } else {
+ return serviceResultBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>optional .CoprocessorServiceResult service_result = 4;</code>
+ *
+ * <pre>
+ * result if this was a coprocessor service call
+ * </pre>
+ */
+ 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;
+ }
+ /**
+ * <code>optional .CoprocessorServiceResult service_result = 4;</code>
+ *
+ * <pre>
+ * result if this was a coprocessor service call
+ * </pre>
+ */
+ 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;
+ }
+ /**
+ * <code>optional .CoprocessorServiceResult service_result = 4;</code>
+ *
+ * <pre>
+ * result if this was a coprocessor service call
+ * </pre>
+ */
+ 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;
+ }
+ /**
+ * <code>optional .CoprocessorServiceResult service_result = 4;</code>
+ *
+ * <pre>
+ * result if this was a coprocessor service call
+ * </pre>
+ */
+ 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;
+ }
+ /**
+ * <code>optional .CoprocessorServiceResult service_result = 4;</code>
+ *
+ * <pre>
+ * result if this was a coprocessor service call
+ * </pre>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResult.Builder getServiceResultBuilder() {
+ bitField0_ |= 0x00000008;
+ onChanged();
+ return getServiceResultFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>optional .CoprocessorServiceResult service_result = 4;</code>
+ *
+ * <pre>
+ * result if this was a coprocessor service call
+ * </pre>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResultOrBuilder getServiceResultOrBuilder() {
+ if (serviceResultBuilder_ != null) {
+ return serviceResultBuilder_.getMessageOrBuilder();
+ } else {
+ return serviceResult_;
+ }
+ }
+ /**
+ * <code>optional .CoprocessorServiceResult service_result = 4;</code>
+ *
+ * <pre>
+ * result if this was a coprocessor service call
+ * </pre>
+ */
+ 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,
Modified: hbase/branches/0.98/hbase-protocol/src/main/protobuf/Client.proto
URL: http://svn.apache.org/viewvc/hbase/branches/0.98/hbase-protocol/src/main/protobuf/Client.proto?rev=1576793&r1=1576792&r2=1576793&view=diff
==============================================================================
--- hbase/branches/0.98/hbase-protocol/src/main/protobuf/Client.proto (original)
+++ hbase/branches/0.98/hbase-protocol/src/main/protobuf/Client.proto Wed Mar 12 15:59:00 2014
@@ -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;
}
/**
Modified: hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java?rev=1576793&r1=1576792&r2=1576793&view=diff
==============================================================================
--- hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java (original)
+++ hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java Wed Mar 12 15:59:00 2014
@@ -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.Ro
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.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CoprocessorClassLoader;
@@ -69,6 +71,8 @@ import org.apache.hadoop.hbase.util.Sort
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;
@@ -611,6 +615,21 @@ public abstract class CoprocessorHost<E
byte[] qualifier, long amount, boolean writeToWAL) throws IOException {
return table.incrementColumnValue(row, family, qualifier, amount, writeToWAL);
}
+
+ @Override
+ public <R extends Message> Map<byte[], R> 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 <R extends Message> void batchCoprocessorService(Descriptors.MethodDescriptor method,
+ Message request, byte[] startKey, byte[] endKey, R responsePrototype,
+ Callback<R> callback) throws ServiceException, Throwable {
+ table.batchCoprocessorService(method, request, startKey, endKey, responsePrototype,
+ callback);
+ }
}
/** The coprocessor */