You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by se...@apache.org on 2013/11/15 05:36:31 UTC
svn commit: r1542168 [1/3] - in /hbase/trunk:
hbase-client/src/main/java/org/apache/hadoop/hbase/client/
hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/
hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/
hbase-common/src/main/j...
Author: sershe
Date: Fri Nov 15 04:36:30 2013
New Revision: 1542168
URL: http://svn.apache.org/r1542168
Log:
HBASE-3787 Increment is non-idempotent but client retries RPC
Added:
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientIdGenerator.java
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NonceGenerator.java
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PerClientRandomNonceGenerator.java
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/OperationConflictException.java
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/ServerNonceManager.java
hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestServerNonceManager.java
Modified:
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Action.java
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Triple.java
hbase/trunk/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
hbase/trunk/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MultiRowMutationProtos.java
hbase/trunk/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/RowProcessorProtos.java
hbase/trunk/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/WALProtos.java
hbase/trunk/hbase-protocol/src/main/protobuf/Client.proto
hbase/trunk/hbase-protocol/src/main/protobuf/MultiRowMutation.proto
hbase/trunk/hbase-protocol/src/main/protobuf/RowProcessor.proto
hbase/trunk/hbase-protocol/src/main/protobuf/WAL.proto
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/client/CoprocessorHConnection.java
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseRowProcessorEndpoint.java
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MultiRowMutationEndpoint.java
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/ReplicationProtbufUtil.java
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerServices.java
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotLogSplitter.java
hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/MockRegionServerServices.java
hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMultiParallel.java
hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java
hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java
Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Action.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Action.java?rev=1542168&r1=1542167&r2=1542168&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Action.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Action.java Fri Nov 15 04:36:30 2013
@@ -19,9 +19,10 @@
package org.apache.hadoop.hbase.client;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
/**
- * A Get, Put or Delete associated with it's region. Used internally by
+ * A Get, Put, Increment, Append, or Delete associated with it's region. Used internally by
* {@link HTable#batch} to associate the action with it's region and maintain
* the index from the original request.
*/
@@ -30,6 +31,7 @@ public class Action<R> implements Compar
// TODO: This class should not be visible outside of the client package.
private Row action;
private int originalIndex;
+ private long nonce = HConstants.NO_NONCE;
public Action(Row action, int originalIndex) {
super();
@@ -37,6 +39,13 @@ public class Action<R> implements Compar
this.originalIndex = originalIndex;
}
+ public void setNonce(long nonce) {
+ this.nonce = nonce;
+ }
+
+ public boolean hasNonce() {
+ return nonce != HConstants.NO_NONCE;
+ }
public Row getAction() {
return action;
@@ -64,4 +73,8 @@ public class Action<R> implements Compar
Action<?> other = (Action<?>) obj;
return compareTo(other) == 0;
}
+
+ public long getNonce() {
+ return nonce;
+ }
}
\ No newline at end of file
Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java?rev=1542168&r1=1542167&r2=1542168&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java Fri Nov 15 04:36:30 2013
@@ -275,6 +275,7 @@ class AsyncProcess<CResult> {
long currentTaskCnt = tasksDone.get();
boolean alreadyLooped = false;
+ NonceGenerator ng = this.hConnection.getNonceGenerator();
do {
if (alreadyLooped){
// if, for whatever reason, we looped, we want to be sure that something has changed.
@@ -302,12 +303,12 @@ class AsyncProcess<CResult> {
it.remove();
} else if (canTakeOperation(loc, regionIncluded, serverIncluded)) {
Action<Row> action = new Action<Row>(r, ++posInList);
+ setNonce(ng, r, action);
retainedActions.add(action);
- addAction(loc, action, actionsByServer);
+ addAction(loc, action, actionsByServer, ng);
it.remove();
}
}
-
} while (retainedActions.isEmpty() && atLeastOne && !hasError());
HConnectionManager.ServerErrorTracker errorsByServer = createServerErrorTracker();
@@ -320,15 +321,21 @@ class AsyncProcess<CResult> {
* @param loc - the destination. Must not be null.
* @param action - the action to add to the multiaction
* @param actionsByServer the multiaction per server
+ * @param ng Nonce generator, or null if no nonces are needed.
*/
private void addAction(HRegionLocation loc, Action<Row> action, Map<HRegionLocation,
- MultiAction<Row>> actionsByServer) {
+ MultiAction<Row>> actionsByServer, NonceGenerator ng) {
final byte[] regionName = loc.getRegionInfo().getRegionName();
MultiAction<Row> multiAction = actionsByServer.get(loc);
if (multiAction == null) {
multiAction = new MultiAction<Row>();
actionsByServer.put(loc, multiAction);
}
+ if (action.hasNonce() && !multiAction.hasNonceGroup()) {
+ // TODO: this code executes for every (re)try, and calls getNonceGroup again
+ // for the same action. It must return the same value across calls.
+ multiAction.setNonceGroup(ng.getNonceGroup());
+ }
multiAction.add(regionName, action);
}
@@ -443,15 +450,22 @@ class AsyncProcess<CResult> {
// The position will be used by the processBatch to match the object array returned.
int posInList = -1;
+ NonceGenerator ng = this.hConnection.getNonceGenerator();
for (Row r : rows) {
posInList++;
Action<Row> action = new Action<Row>(r, posInList);
+ setNonce(ng, r, action);
actions.add(action);
}
HConnectionManager.ServerErrorTracker errorsByServer = createServerErrorTracker();
submit(actions, actions, 1, errorsByServer);
}
+ private void setNonce(NonceGenerator ng, Row r, Action<Row> action) {
+ if (!(r instanceof Append) && !(r instanceof Increment)) return;
+ action.setNonce(ng.newNonce()); // Action handles NO_NONCE, so it's ok if ng is disabled.
+ }
+
/**
* Group a list of actions per region servers, and send them. The created MultiActions are
@@ -473,10 +487,11 @@ class AsyncProcess<CResult> {
final Map<HRegionLocation, MultiAction<Row>> actionsByServer =
new HashMap<HRegionLocation, MultiAction<Row>>();
+ NonceGenerator ng = this.hConnection.getNonceGenerator();
for (Action<Row> action : currentActions) {
HRegionLocation loc = findDestLocation(action.getAction(), action.getOriginalIndex());
if (loc != null) {
- addAction(loc, action, actionsByServer);
+ addAction(loc, action, actionsByServer, ng);
}
}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientIdGenerator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientIdGenerator.java?rev=1542168&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientIdGenerator.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClientIdGenerator.java Fri Nov 15 04:36:30 2013
@@ -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 java.io.IOException;
+import java.lang.management.ManagementFactory;
+import java.net.Inet4Address;
+import java.net.Inet6Address;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.util.Enumeration;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * The class that is able to determine some unique strings for the client,
+ * such as an IP address, PID, and composite deterministic ID.
+ */
+@InterfaceAudience.Private
+class ClientIdGenerator {
+ static final Log LOG = LogFactory.getLog(ClientIdGenerator.class);
+
+ /**
+ * @return a unique ID incorporating IP address, PID, TID and timer. Might be an overkill...
+ * Note though that new UUID in java by default is just a random number.
+ */
+ public static byte[] generateClientId() {
+ byte[] selfBytes = getIpAddressBytes();
+ Long pid = getPid();
+ long tid = Thread.currentThread().getId();
+ long ts = System.currentTimeMillis();
+
+ byte[] id = new byte[selfBytes.length + ((pid != null ? 1 : 0) + 2) * Bytes.SIZEOF_LONG];
+ int offset = Bytes.putBytes(id, 0, selfBytes, 0, selfBytes.length);
+ if (pid != null) {
+ offset = Bytes.putLong(id, offset, pid);
+ }
+ offset = Bytes.putLong(id, offset, tid);
+ offset = Bytes.putLong(id, offset, ts);
+ assert offset == id.length;
+ return id;
+ }
+
+ /**
+ * @return PID of the current process, if it can be extracted from JVM name, or null.
+ */
+ public static Long getPid() {
+ String name = ManagementFactory.getRuntimeMXBean().getName();
+ String[] nameParts = name.split("@");
+ if (nameParts.length == 2) { // 12345@somewhere
+ try {
+ return Long.parseLong(nameParts[0]);
+ } catch (NumberFormatException ex) {
+ LOG.warn("Failed to get PID from [" + name + "]", ex);
+ }
+ } else {
+ LOG.warn("Don't know how to get PID from [" + name + "]");
+ }
+ return null;
+ }
+
+ /**
+ * @return Some IPv4/IPv6 address available on the current machine that is up, not virtual
+ * and not a loopback address. Empty array if none can be found or error occured.
+ */
+ public static byte[] getIpAddressBytes() {
+ try {
+ // Before we connect somewhere, we cannot be sure about what we'd be bound to; however,
+ // we only connect when the message where client ID is, is long constructed. Thus,
+ // just use whichever IP address we can find.
+ Enumeration<NetworkInterface> interfaces = NetworkInterface.getNetworkInterfaces();
+ while (interfaces.hasMoreElements()) {
+ NetworkInterface current = interfaces.nextElement();
+ if (!current.isUp() || current.isLoopback() || current.isVirtual()) continue;
+ Enumeration<InetAddress> addresses = current.getInetAddresses();
+ while (addresses.hasMoreElements()) {
+ InetAddress addr = addresses.nextElement();
+ if (addr.isLoopbackAddress()) continue;
+ if (addr instanceof Inet4Address || addr instanceof Inet6Address) {
+ return addr.getAddress();
+ }
+ }
+ }
+ } catch (IOException ex) {
+ LOG.warn("Failed to get IP address bytes", ex);
+ }
+ return new byte[0];
+ }
+}
Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java?rev=1542168&r1=1542167&r2=1542168&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnection.java Fri Nov 15 04:36:30 2013
@@ -515,4 +515,9 @@ public interface HConnection extends Abo
* @return true if the server is known as dead, false otherwise.
*/
boolean isDeadServer(ServerName serverName);
+
+ /**
+ * @return Nonce generator for this HConnection; may be null if disabled in configuration.
+ */
+ public NonceGenerator getNonceGenerator();
}
\ No newline at end of file
Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java?rev=1542168&r1=1542167&r2=1542168&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java Fri Nov 15 04:36:30 2013
@@ -124,6 +124,7 @@ import org.apache.hadoop.hbase.zookeeper
import org.apache.hadoop.ipc.RemoteException;
import org.apache.zookeeper.KeeperException;
+import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.BlockingRpcChannel;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
@@ -197,6 +198,7 @@ public class HConnectionManager {
static final Log LOG = LogFactory.getLog(HConnectionManager.class);
public static final String RETRIES_BY_SERVER_KEY = "hbase.client.retries.by.server";
+ private static final String CLIENT_NONCES_ENABLED_KEY = "hbase.client.nonces.enabled";
// An LRU Map of HConnectionKey -> HConnection (TableServer). All
// access must be synchronized. This map is not private because tests
@@ -205,6 +207,14 @@ public class HConnectionManager {
public static final int MAX_CACHED_CONNECTION_INSTANCES;
+ /**
+ * Global nonceGenerator shared per client.Currently there's no reason to limit its scope.
+ * Once it's set under nonceGeneratorCreateLock, it is never unset or changed.
+ */
+ private static volatile NonceGenerator nonceGenerator = null;
+ /** The nonce generator lock. Only taken when creating HConnection, which gets a private copy. */
+ private static Object nonceGeneratorCreateLock = new Object();
+
static {
// We set instances to one more than the value specified for {@link
// HConstants#ZOOKEEPER_MAX_CLIENT_CNXNS}. By default, the zk default max
@@ -230,6 +240,20 @@ public class HConnectionManager {
}
/**
+ * @param conn The connection for which to replace the generator.
+ * @param cnm Replaces the nonce generator used, for testing.
+ * @return old nonce generator.
+ */
+ @VisibleForTesting
+ public static NonceGenerator injectNonceGeneratorForTesting(
+ HConnection conn, NonceGenerator cnm) {
+ NonceGenerator ng = conn.getNonceGenerator();
+ LOG.warn("Nonce generator is being replaced by test code for " + cnm.getClass().getName());
+ ((HConnectionImplementation)conn).nonceGenerator = cnm;
+ return ng;
+ }
+
+ /**
* Get the connection that goes with the passed <code>conf</code> configuration instance.
* If no current connection exists, method creates a new connection and keys it using
* connection-specific properties from the passed {@link Configuration}; see
@@ -547,6 +571,7 @@ public class HConnectionManager {
private final long pause;
private final int numTries;
final int rpcTimeout;
+ private NonceGenerator nonceGenerator = null;
private final int prefetchRegionLimit;
private volatile boolean closed;
@@ -661,6 +686,17 @@ public class HConnectionManager {
}
}
+ /** Dummy nonce generator for disabled nonces. */
+ private static class NoNonceGenerator implements NonceGenerator {
+ @Override
+ public long getNonceGroup() {
+ return HConstants.NO_NONCE;
+ }
+ @Override
+ public long newNonce() {
+ return HConstants.NO_NONCE;
+ }
+ }
/**
* For tests.
@@ -675,6 +711,17 @@ public class HConnectionManager {
this.rpcTimeout = conf.getInt(
HConstants.HBASE_RPC_TIMEOUT_KEY,
HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
+ if (conf.getBoolean(CLIENT_NONCES_ENABLED_KEY, true)) {
+ synchronized (HConnectionManager.nonceGeneratorCreateLock) {
+ if (HConnectionManager.nonceGenerator == null) {
+ HConnectionManager.nonceGenerator = new PerClientRandomNonceGenerator();
+ }
+ this.nonceGenerator = HConnectionManager.nonceGenerator;
+ }
+ } else {
+ this.nonceGenerator = new NoNonceGenerator();
+ }
+
this.prefetchRegionLimit = conf.getInt(
HConstants.HBASE_CLIENT_PREFETCH_LIMIT,
HConstants.DEFAULT_HBASE_CLIENT_PREFETCH_LIMIT);
@@ -2617,6 +2664,11 @@ public class HConnectionManager {
return getHTableDescriptorsByTableName(tableNames);
}
+ @Override
+ public NonceGenerator getNonceGenerator() {
+ return this.nonceGenerator;
+ }
+
/**
* Connects to the master to get the table descriptor.
* @param tableName table name
Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java?rev=1542168&r1=1542167&r2=1542168&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java Fri Nov 15 04:36:30 2013
@@ -986,12 +986,15 @@ public class HTable implements HTableInt
throw new IOException(
"Invalid arguments to append, no columns specified");
}
+
+ NonceGenerator ng = this.connection.getNonceGenerator();
+ final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
RegionServerCallable<Result> callable =
new RegionServerCallable<Result>(this.connection, getName(), append.getRow()) {
public Result call() throws IOException {
try {
MutateRequest request = RequestConverter.buildMutateRequest(
- getLocation().getRegionInfo().getRegionName(), append);
+ getLocation().getRegionInfo().getRegionName(), append, nonceGroup, nonce);
PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
rpcController.setPriority(getTableName());
MutateResponse response = getStub().mutate(rpcController, request);
@@ -1014,19 +1017,21 @@ public class HTable implements HTableInt
throw new IOException(
"Invalid arguments to increment, no columns specified");
}
+ NonceGenerator ng = this.connection.getNonceGenerator();
+ final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
getName(), increment.getRow()) {
public Result call() throws IOException {
try {
MutateRequest request = RequestConverter.buildMutateRequest(
- getLocation().getRegionInfo().getRegionName(), increment);
- PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
- rpcController.setPriority(getTableName());
- MutateResponse response = getStub().mutate(rpcController, request);
- return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
- } catch (ServiceException se) {
- throw ProtobufUtil.getRemoteException(se);
- }
+ getLocation().getRegionInfo().getRegionName(), increment, nonceGroup, nonce);
+ PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
+ rpcController.setPriority(getTableName());
+ MutateResponse response = getStub().mutate(rpcController, request);
+ return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
+ } catch (ServiceException se) {
+ throw ProtobufUtil.getRemoteException(se);
+ }
}
};
return rpcCallerFactory.<Result> newCaller().callWithRetries(callable, this.operationTimeout);
@@ -1074,13 +1079,15 @@ public class HTable implements HTableInt
"Invalid arguments to incrementColumnValue", npe);
}
+ NonceGenerator ng = this.connection.getNonceGenerator();
+ final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
RegionServerCallable<Long> callable =
new RegionServerCallable<Long>(connection, getName(), row) {
public Long call() throws IOException {
try {
- MutateRequest request = RequestConverter.buildMutateRequest(
+ MutateRequest request = RequestConverter.buildIncrementRequest(
getLocation().getRegionInfo().getRegionName(), row, family,
- qualifier, amount, durability);
+ qualifier, amount, durability, nonceGroup, nonce);
PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
rpcController.setPriority(getTableName());
MutateResponse response = getStub().mutate(rpcController, request);
Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java?rev=1542168&r1=1542167&r2=1542168&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiAction.java Fri Nov 15 04:36:30 2013
@@ -25,6 +25,7 @@ import java.util.Set;
import java.util.TreeMap;
import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.util.Bytes;
/**
@@ -39,6 +40,8 @@ public final class MultiAction<R> {
public Map<byte[], List<Action<R>>> actions =
new TreeMap<byte[], List<Action<R>>>(Bytes.BYTES_COMPARATOR);
+ private long nonceGroup = HConstants.NO_NONCE;
+
public MultiAction() {
super();
}
@@ -73,6 +76,10 @@ public final class MultiAction<R> {
rsActions.add(a);
}
+ public void setNonceGroup(long nonceGroup) {
+ this.nonceGroup = nonceGroup;
+ }
+
public Set<byte[]> getRegions() {
return actions.keySet();
}
@@ -87,4 +94,12 @@ public final class MultiAction<R> {
}
return res;
}
+
+ public boolean hasNonceGroup() {
+ return nonceGroup != HConstants.NO_NONCE;
+ }
+
+ public long getNonceGroup() {
+ return this.nonceGroup;
+ }
}
\ No newline at end of file
Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java?rev=1542168&r1=1542167&r2=1542168&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/MultiServerCallable.java Fri Nov 15 04:36:30 2013
@@ -19,12 +19,14 @@ package org.apache.hadoop.hbase.client;
import java.io.IOException;
import java.util.ArrayList;
+import java.util.Collections;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
@@ -71,6 +73,10 @@ class MultiServerCallable<R> extends Reg
MutationProto.Builder mutationBuilder = MutationProto.newBuilder();
List<CellScannable> cells = null;
// The multi object is a list of Actions by region. Iterate by region.
+ long nonceGroup = multiAction.getNonceGroup();
+ if (nonceGroup != HConstants.NO_NONCE) {
+ multiRequestBuilder.setNonceGroup(nonceGroup);
+ }
for (Map.Entry<byte[], List<Action<R>>> e: this.multiAction.actions.entrySet()) {
final byte [] regionName = e.getKey();
final List<Action<R>> actions = e.getValue();
@@ -92,6 +98,7 @@ class MultiServerCallable<R> extends Reg
}
multiRequestBuilder.addRegionAction(regionActionBuilder.build());
}
+
// Controller optionally carries cell data over the proxy/service boundary and also
// optionally ferries cell response data back out again.
PayloadCarryingRpcController controller = new PayloadCarryingRpcController(cells);
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NonceGenerator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NonceGenerator.java?rev=1542168&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NonceGenerator.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/NonceGenerator.java Fri Nov 15 04:36:30 2013
@@ -0,0 +1,37 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * NonceGenerator interface.
+ * In general, nonce group is an ID (one per client, or region+client, or whatever) that
+ * could be used to reduce collision potential, or be used by compatible server nonce manager
+ * to optimize nonce storage and removal. See HBASE-3787.
+ */
+@InterfaceAudience.Private
+public interface NonceGenerator {
+
+ /** @return the nonce group (client ID) of this client manager. */
+ public long getNonceGroup();
+
+ /** @return New nonce. */
+ public long newNonce();
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PerClientRandomNonceGenerator.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PerClientRandomNonceGenerator.java?rev=1542168&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PerClientRandomNonceGenerator.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/client/PerClientRandomNonceGenerator.java Fri Nov 15 04:36:30 2013
@@ -0,0 +1,34 @@
+package org.apache.hadoop.hbase.client;
+
+import java.util.Arrays;
+import java.util.Random;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.HConstants;
+
+/**
+ * NonceGenerator implementation that uses client ID hash + random int as nonce group,
+ * and random numbers as nonces.
+ */
+@InterfaceAudience.Private
+public class PerClientRandomNonceGenerator implements NonceGenerator {
+ private final Random rdm = new Random();
+ private final long clientId;
+
+ public PerClientRandomNonceGenerator() {
+ byte[] clientIdBase = ClientIdGenerator.generateClientId();
+ this.clientId = (((long)Arrays.hashCode(clientIdBase)) << 32) + rdm.nextInt();
+ }
+
+ public long getNonceGroup() {
+ return this.clientId;
+ }
+
+ public long newNonce() {
+ long result = HConstants.NO_NONCE;
+ do {
+ result = rdm.nextLong();
+ } while (result == HConstants.NO_NONCE);
+ return result;
+ }
+}
Added: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/OperationConflictException.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/OperationConflictException.java?rev=1542168&view=auto
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/OperationConflictException.java (added)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/exceptions/OperationConflictException.java Fri Nov 15 04:36:30 2013
@@ -0,0 +1,49 @@
+/**
+ *
+ * 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.exceptions;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+
+/**
+ * The exception that is thrown if there's duplicate execution of non-idempotent operation.
+ * Client should not retry; may use "get" to get the desired value.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class OperationConflictException extends DoNotRetryIOException {
+ private static final long serialVersionUID = -8930333627489862872L;
+
+ public OperationConflictException() {
+ super();
+ }
+
+ public OperationConflictException(String message) {
+ super(message);
+ }
+
+ public OperationConflictException(Throwable cause) {
+ super(cause);
+ }
+
+ public OperationConflictException(String message, Throwable cause) {
+ super(message, cause);
+ }
+}
Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java?rev=1542168&r1=1542167&r2=1542168&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java Fri Nov 15 04:36:30 2013
@@ -988,11 +988,14 @@ public final class ProtobufUtil {
* @param increment
* @return the converted mutate
*/
- public static MutationProto toMutation(final Increment increment,
- final MutationProto.Builder builder) {
+ public static MutationProto toMutation(
+ final Increment increment, final MutationProto.Builder builder, long nonce) {
builder.setRow(ZeroCopyLiteralByteString.wrap(increment.getRow()));
builder.setMutateType(MutationType.INCREMENT);
builder.setDurability(toDurability(increment.getDurability()));
+ if (nonce != HConstants.NO_NONCE) {
+ builder.setNonce(nonce);
+ }
TimeRange timeRange = increment.getTimeRange();
if (!timeRange.isAllTime()) {
HBaseProtos.TimeRange.Builder timeRangeBuilder =
@@ -1035,6 +1038,11 @@ public final class ProtobufUtil {
return builder.build();
}
+ public static MutationProto toMutation(final MutationType type, final Mutation mutation)
+ throws IOException {
+ return toMutation(type, mutation, HConstants.NO_NONCE);
+ }
+
/**
* Create a protocol buffer Mutate based on a client Mutation
*
@@ -1043,15 +1051,23 @@ public final class ProtobufUtil {
* @return a protobuf'd Mutation
* @throws IOException
*/
- public static MutationProto toMutation(final MutationType type, final Mutation mutation)
- throws IOException {
- return toMutation(type, mutation, MutationProto.newBuilder());
+ public static MutationProto toMutation(final MutationType type, final Mutation mutation,
+ final long nonce) throws IOException {
+ return toMutation(type, mutation, MutationProto.newBuilder(), nonce);
+ }
+
+ public static MutationProto toMutation(final MutationType type, final Mutation mutation,
+ MutationProto.Builder builder) throws IOException {
+ return toMutation(type, mutation, builder, HConstants.NO_NONCE);
}
public static MutationProto toMutation(final MutationType type, final Mutation mutation,
- MutationProto.Builder builder)
+ MutationProto.Builder builder, long nonce)
throws IOException {
builder = getMutationBuilderAndSetCommonFields(type, mutation, builder);
+ if (nonce != HConstants.NO_NONCE) {
+ builder.setNonce(nonce);
+ }
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
for (Map.Entry<byte[],List<Cell>> family: mutation.getFamilyCellMap().entrySet()) {
@@ -1078,6 +1094,11 @@ public final class ProtobufUtil {
return builder.build();
}
+ public static MutationProto toMutationNoData(final MutationType type, final Mutation mutation,
+ final MutationProto.Builder builder) throws IOException {
+ return toMutationNoData(type, mutation, builder, HConstants.NO_NONCE);
+ }
+
/**
* Create a protocol buffer MutationProto based on a client Mutation. Does NOT include data.
* Understanding is that the Cell will be transported other than via protobuf.
@@ -1087,10 +1108,12 @@ public final class ProtobufUtil {
* @throws IOException
*/
public static MutationProto toMutationNoData(final MutationType type, final Mutation mutation,
- final MutationProto.Builder builder)
- throws IOException {
+ final MutationProto.Builder builder, long nonce) throws IOException {
getMutationBuilderAndSetCommonFields(type, mutation, builder);
builder.setAssociatedCellCount(mutation.size());
+ if (nonce != HConstants.NO_NONCE) {
+ builder.setNonce(nonce);
+ }
return builder.build();
}
Modified: hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java?rev=1542168&r1=1542167&r2=1542168&view=diff
==============================================================================
--- hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (original)
+++ hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java Fri Nov 15 04:36:30 2013
@@ -22,6 +22,8 @@ import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CellScannable;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
@@ -172,9 +174,9 @@ public final class RequestConverter {
* @param durability
* @return a mutate request
*/
- public static MutateRequest buildMutateRequest(
- final byte[] regionName, final byte[] row, final byte[] family,
- final byte [] qualifier, final long amount, final Durability durability) {
+ public static MutateRequest buildIncrementRequest(
+ final byte[] regionName, final byte[] row, final byte[] family, final byte[] qualifier,
+ final long amount, final Durability durability, long nonceGroup, long nonce) {
MutateRequest.Builder builder = MutateRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
@@ -191,7 +193,13 @@ public final class RequestConverter {
valueBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier));
columnBuilder.addQualifierValue(valueBuilder.build());
mutateBuilder.addColumnValue(columnBuilder.build());
+ if (nonce != HConstants.NO_NONCE) {
+ mutateBuilder.setNonce(nonce);
+ }
builder.setMutation(mutateBuilder.build());
+ if (nonceGroup != HConstants.NO_NONCE) {
+ builder.setNonceGroup(nonceGroup);
+ }
return builder.build();
}
@@ -278,14 +286,17 @@ public final class RequestConverter {
* @return a mutate request
* @throws IOException
*/
- public static MutateRequest buildMutateRequest(
- final byte[] regionName, final Append append) throws IOException {
+ public static MutateRequest buildMutateRequest(final byte[] regionName,
+ final Append append, long nonceGroup, long nonce) throws IOException {
MutateRequest.Builder builder = MutateRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
+ if (nonce != HConstants.NO_NONCE && nonceGroup != HConstants.NO_NONCE) {
+ builder.setNonceGroup(nonceGroup);
+ }
builder.setMutation(ProtobufUtil.toMutation(MutationType.APPEND, append,
- MutationProto.newBuilder()));
+ MutationProto.newBuilder(), nonce));
return builder.build();
}
@@ -296,13 +307,16 @@ public final class RequestConverter {
* @param increment
* @return a mutate request
*/
- public static MutateRequest buildMutateRequest(
- final byte[] regionName, final Increment increment) {
+ public static MutateRequest buildMutateRequest(final byte[] regionName,
+ final Increment increment, final long nonceGroup, final long nonce) {
MutateRequest.Builder builder = MutateRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
- builder.setMutation(ProtobufUtil.toMutation(increment, MutationProto.newBuilder()));
+ if (nonce != HConstants.NO_NONCE && nonceGroup != HConstants.NO_NONCE) {
+ builder.setNonceGroup(nonceGroup);
+ }
+ builder.setMutation(ProtobufUtil.toMutation(increment, MutationProto.newBuilder(), nonce));
return builder.build();
}
@@ -499,8 +513,7 @@ public final class RequestConverter {
public static <R> RegionAction.Builder buildRegionAction(final byte[] regionName,
final List<Action<R>> actions, final RegionAction.Builder regionActionBuilder,
final ClientProtos.Action.Builder actionBuilder,
- final MutationProto.Builder mutationBuilder)
- throws IOException {
+ final MutationProto.Builder mutationBuilder) throws IOException {
for (Action<R> action: actions) {
Row row = action.getAction();
actionBuilder.clear();
@@ -516,11 +529,11 @@ public final class RequestConverter {
regionActionBuilder.addAction(actionBuilder.
setMutation(ProtobufUtil.toMutation(MutationType.DELETE, (Delete)row, mutationBuilder)));
} else if (row instanceof Append) {
- regionActionBuilder.addAction(actionBuilder.
- setMutation(ProtobufUtil.toMutation(MutationType.APPEND, (Append)row, mutationBuilder)));
+ regionActionBuilder.addAction(actionBuilder.setMutation(ProtobufUtil.toMutation(
+ MutationType.APPEND, (Append)row, mutationBuilder, action.getNonce())));
} else if (row instanceof Increment) {
- regionActionBuilder.addAction(actionBuilder.
- setMutation(ProtobufUtil.toMutation((Increment)row, mutationBuilder)));
+ regionActionBuilder.addAction(actionBuilder.setMutation(
+ ProtobufUtil.toMutation((Increment)row, mutationBuilder, action.getNonce())));
} else if (row instanceof RowMutations) {
throw new UnsupportedOperationException("No RowMutations in multi calls; use mutateRow");
} else {
@@ -550,10 +563,9 @@ public final class RequestConverter {
final List<Action<R>> actions, final List<CellScannable> cells,
final RegionAction.Builder regionActionBuilder,
final ClientProtos.Action.Builder actionBuilder,
- final MutationProto.Builder mutationBuilder)
- throws IOException {
- RegionAction.Builder builder =
- getRegionActionBuilderWithRegion(RegionAction.newBuilder(), regionName);
+ final MutationProto.Builder mutationBuilder) throws IOException {
+ RegionAction.Builder builder = getRegionActionBuilderWithRegion(
+ RegionAction.newBuilder(), regionName);
for (Action<R> action: actions) {
Row row = action.getAction();
actionBuilder.clear();
@@ -586,13 +598,13 @@ public final class RequestConverter {
} else if (row instanceof Append) {
Append a = (Append)row;
cells.add(a);
- builder.addAction(actionBuilder.
- setMutation(ProtobufUtil.toMutationNoData(MutationType.APPEND, a, mutationBuilder)));
+ builder.addAction(actionBuilder.setMutation(ProtobufUtil.toMutationNoData(
+ MutationType.APPEND, a, mutationBuilder, action.getNonce())));
} else if (row instanceof Increment) {
Increment i = (Increment)row;
cells.add(i);
- builder.addAction(actionBuilder.
- setMutation(ProtobufUtil.toMutationNoData(MutationType.INCREMENT, i, mutationBuilder)));
+ builder.addAction(actionBuilder.setMutation(ProtobufUtil.toMutationNoData(
+ MutationType.INCREMENT, i, mutationBuilder, action.getNonce())));
} else if (row instanceof RowMutations) {
continue; // ignore RowMutations
} else {
Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java?rev=1542168&r1=1542167&r2=1542168&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java Fri Nov 15 04:36:30 2013
@@ -743,6 +743,9 @@ public final class HConstants {
public static final String HBASE_REGION_SPLIT_POLICY_KEY =
"hbase.regionserver.region.split.policy";
+ /** Whether nonces are enabled; default is true. */
+ public static String HBASE_RS_NONCES_ENABLED = "hbase.regionserver.nonces.enabled";
+
/**
* Configuration key for the size of the block cache
*/
@@ -934,6 +937,8 @@ public final class HConstants {
public static final String STATUS_MULTICAST_PORT = "hbase.status.multicast.port";
public static final int DEFAULT_STATUS_MULTICAST_PORT = 60100;
+ public static final long NO_NONCE = 0;
+
private HConstants() {
// Can't be instantiated with this ctor.
}
Modified: hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Triple.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Triple.java?rev=1542168&r1=1542167&r2=1542168&view=diff
==============================================================================
--- hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Triple.java (original)
+++ hbase/trunk/hbase-common/src/main/java/org/apache/hadoop/hbase/util/Triple.java Fri Nov 15 04:36:30 2013
@@ -35,6 +35,11 @@ public class Triple<A, B, C> {
this.third = third;
}
+ // ctor cannot infer types w/o warning but a method can.
+ public static <A, B, C> Triple<A, B, C> create(A first, B second, C third) {
+ return new Triple<A, B, C>(first, second, third);
+ }
+
public int hashCode() {
int hashFirst = (first != null ? first.hashCode() : 0);
int hashSecond = (second != null ? second.hashCode() : 0);
Modified: hbase/trunk/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java?rev=1542168&r1=1542167&r2=1542168&view=diff
==============================================================================
--- hbase/trunk/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java (original)
+++ hbase/trunk/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/ClientProtos.java Fri Nov 15 04:36:30 2013
@@ -6487,6 +6487,16 @@ public final class ClientProtos {
* </pre>
*/
int getAssociatedCellCount();
+
+ // optional uint64 nonce = 9;
+ /**
+ * <code>optional uint64 nonce = 9;</code>
+ */
+ boolean hasNonce();
+ /**
+ * <code>optional uint64 nonce = 9;</code>
+ */
+ long getNonce();
}
/**
* Protobuf type {@code MutationProto}
@@ -6614,6 +6624,11 @@ public final class ClientProtos {
associatedCellCount_ = input.readInt32();
break;
}
+ case 72: {
+ bitField0_ |= 0x00000040;
+ nonce_ = input.readUInt64();
+ break;
+ }
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -8801,6 +8816,22 @@ public final class ClientProtos {
return associatedCellCount_;
}
+ // optional uint64 nonce = 9;
+ public static final int NONCE_FIELD_NUMBER = 9;
+ private long nonce_;
+ /**
+ * <code>optional uint64 nonce = 9;</code>
+ */
+ public boolean hasNonce() {
+ return ((bitField0_ & 0x00000040) == 0x00000040);
+ }
+ /**
+ * <code>optional uint64 nonce = 9;</code>
+ */
+ public long getNonce() {
+ return nonce_;
+ }
+
private void initFields() {
row_ = com.google.protobuf.ByteString.EMPTY;
mutateType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType.APPEND;
@@ -8810,6 +8841,7 @@ public final class ClientProtos {
durability_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Durability.USE_DEFAULT;
timeRange_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.getDefaultInstance();
associatedCellCount_ = 0;
+ nonce_ = 0L;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -8859,6 +8891,9 @@ public final class ClientProtos {
if (((bitField0_ & 0x00000020) == 0x00000020)) {
output.writeInt32(8, associatedCellCount_);
}
+ if (((bitField0_ & 0x00000040) == 0x00000040)) {
+ output.writeUInt64(9, nonce_);
+ }
getUnknownFields().writeTo(output);
}
@@ -8900,6 +8935,10 @@ public final class ClientProtos {
size += com.google.protobuf.CodedOutputStream
.computeInt32Size(8, associatedCellCount_);
}
+ if (((bitField0_ & 0x00000040) == 0x00000040)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(9, nonce_);
+ }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@@ -8957,6 +8996,11 @@ public final class ClientProtos {
result = result && (getAssociatedCellCount()
== other.getAssociatedCellCount());
}
+ result = result && (hasNonce() == other.hasNonce());
+ if (hasNonce()) {
+ result = result && (getNonce()
+ == other.getNonce());
+ }
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@@ -9002,6 +9046,10 @@ public final class ClientProtos {
hash = (37 * hash) + ASSOCIATED_CELL_COUNT_FIELD_NUMBER;
hash = (53 * hash) + getAssociatedCellCount();
}
+ if (hasNonce()) {
+ hash = (37 * hash) + NONCE_FIELD_NUMBER;
+ hash = (53 * hash) + hashLong(getNonce());
+ }
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@@ -9151,6 +9199,8 @@ public final class ClientProtos {
bitField0_ = (bitField0_ & ~0x00000040);
associatedCellCount_ = 0;
bitField0_ = (bitField0_ & ~0x00000080);
+ nonce_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000100);
return this;
}
@@ -9225,6 +9275,10 @@ public final class ClientProtos {
to_bitField0_ |= 0x00000020;
}
result.associatedCellCount_ = associatedCellCount_;
+ if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
+ to_bitField0_ |= 0x00000040;
+ }
+ result.nonce_ = nonce_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@@ -9311,6 +9365,9 @@ public final class ClientProtos {
if (other.hasAssociatedCellCount()) {
setAssociatedCellCount(other.getAssociatedCellCount());
}
+ if (other.hasNonce()) {
+ setNonce(other.getNonce());
+ }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@@ -10202,6 +10259,39 @@ public final class ClientProtos {
return this;
}
+ // optional uint64 nonce = 9;
+ private long nonce_ ;
+ /**
+ * <code>optional uint64 nonce = 9;</code>
+ */
+ public boolean hasNonce() {
+ return ((bitField0_ & 0x00000100) == 0x00000100);
+ }
+ /**
+ * <code>optional uint64 nonce = 9;</code>
+ */
+ public long getNonce() {
+ return nonce_;
+ }
+ /**
+ * <code>optional uint64 nonce = 9;</code>
+ */
+ public Builder setNonce(long value) {
+ bitField0_ |= 0x00000100;
+ nonce_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional uint64 nonce = 9;</code>
+ */
+ public Builder clearNonce() {
+ bitField0_ = (bitField0_ & ~0x00000100);
+ nonce_ = 0L;
+ onChanged();
+ return this;
+ }
+
// @@protoc_insertion_point(builder_scope:MutationProto)
}
@@ -10257,6 +10347,16 @@ public final class ClientProtos {
* <code>optional .Condition condition = 3;</code>
*/
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ConditionOrBuilder getConditionOrBuilder();
+
+ // optional uint64 nonce_group = 4;
+ /**
+ * <code>optional uint64 nonce_group = 4;</code>
+ */
+ boolean hasNonceGroup();
+ /**
+ * <code>optional uint64 nonce_group = 4;</code>
+ */
+ long getNonceGroup();
}
/**
* Protobuf type {@code MutateRequest}
@@ -10359,6 +10459,11 @@ public final class ClientProtos {
bitField0_ |= 0x00000004;
break;
}
+ case 32: {
+ bitField0_ |= 0x00000008;
+ nonceGroup_ = input.readUInt64();
+ break;
+ }
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -10465,10 +10570,27 @@ public final class ClientProtos {
return condition_;
}
+ // optional uint64 nonce_group = 4;
+ public static final int NONCE_GROUP_FIELD_NUMBER = 4;
+ private long nonceGroup_;
+ /**
+ * <code>optional uint64 nonce_group = 4;</code>
+ */
+ public boolean hasNonceGroup() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional uint64 nonce_group = 4;</code>
+ */
+ public long getNonceGroup() {
+ return nonceGroup_;
+ }
+
private void initFields() {
region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance();
mutation_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance();
condition_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Condition.getDefaultInstance();
+ nonceGroup_ = 0L;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -10513,6 +10635,9 @@ public final class ClientProtos {
if (((bitField0_ & 0x00000004) == 0x00000004)) {
output.writeMessage(3, condition_);
}
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ output.writeUInt64(4, nonceGroup_);
+ }
getUnknownFields().writeTo(output);
}
@@ -10534,6 +10659,10 @@ public final class ClientProtos {
size += com.google.protobuf.CodedOutputStream
.computeMessageSize(3, condition_);
}
+ if (((bitField0_ & 0x00000008) == 0x00000008)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(4, nonceGroup_);
+ }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@@ -10572,6 +10701,11 @@ public final class ClientProtos {
result = result && getCondition()
.equals(other.getCondition());
}
+ result = result && (hasNonceGroup() == other.hasNonceGroup());
+ if (hasNonceGroup()) {
+ result = result && (getNonceGroup()
+ == other.getNonceGroup());
+ }
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@@ -10597,6 +10731,10 @@ public final class ClientProtos {
hash = (37 * hash) + CONDITION_FIELD_NUMBER;
hash = (53 * hash) + getCondition().hashCode();
}
+ if (hasNonceGroup()) {
+ hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
+ hash = (53 * hash) + hashLong(getNonceGroup());
+ }
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@@ -10738,6 +10876,8 @@ public final class ClientProtos {
conditionBuilder_.clear();
}
bitField0_ = (bitField0_ & ~0x00000004);
+ nonceGroup_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000008);
return this;
}
@@ -10790,6 +10930,10 @@ public final class ClientProtos {
} else {
result.condition_ = conditionBuilder_.build();
}
+ if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+ to_bitField0_ |= 0x00000008;
+ }
+ result.nonceGroup_ = nonceGroup_;
result.bitField0_ = to_bitField0_;
onBuilt();
return result;
@@ -10815,6 +10959,9 @@ public final class ClientProtos {
if (other.hasCondition()) {
mergeCondition(other.getCondition());
}
+ if (other.hasNonceGroup()) {
+ setNonceGroup(other.getNonceGroup());
+ }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@@ -11215,6 +11362,39 @@ public final class ClientProtos {
return conditionBuilder_;
}
+ // optional uint64 nonce_group = 4;
+ private long nonceGroup_ ;
+ /**
+ * <code>optional uint64 nonce_group = 4;</code>
+ */
+ public boolean hasNonceGroup() {
+ return ((bitField0_ & 0x00000008) == 0x00000008);
+ }
+ /**
+ * <code>optional uint64 nonce_group = 4;</code>
+ */
+ public long getNonceGroup() {
+ return nonceGroup_;
+ }
+ /**
+ * <code>optional uint64 nonce_group = 4;</code>
+ */
+ public Builder setNonceGroup(long value) {
+ bitField0_ |= 0x00000008;
+ nonceGroup_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional uint64 nonce_group = 4;</code>
+ */
+ public Builder clearNonceGroup() {
+ bitField0_ = (bitField0_ & ~0x00000008);
+ nonceGroup_ = 0L;
+ onChanged();
+ return this;
+ }
+
// @@protoc_insertion_point(builder_scope:MutateRequest)
}
@@ -25450,6 +25630,16 @@ public final class ClientProtos {
*/
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionOrBuilder getRegionActionOrBuilder(
int index);
+
+ // optional uint64 nonceGroup = 2;
+ /**
+ * <code>optional uint64 nonceGroup = 2;</code>
+ */
+ boolean hasNonceGroup();
+ /**
+ * <code>optional uint64 nonceGroup = 2;</code>
+ */
+ long getNonceGroup();
}
/**
* Protobuf type {@code MultiRequest}
@@ -25519,6 +25709,11 @@ public final class ClientProtos {
regionAction_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.PARSER, extensionRegistry));
break;
}
+ case 16: {
+ bitField0_ |= 0x00000001;
+ nonceGroup_ = input.readUInt64();
+ break;
+ }
}
}
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -25561,6 +25756,7 @@ public final class ClientProtos {
return PARSER;
}
+ private int bitField0_;
// repeated .RegionAction regionAction = 1;
public static final int REGIONACTION_FIELD_NUMBER = 1;
private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction> regionAction_;
@@ -25597,8 +25793,25 @@ public final class ClientProtos {
return regionAction_.get(index);
}
+ // optional uint64 nonceGroup = 2;
+ public static final int NONCEGROUP_FIELD_NUMBER = 2;
+ private long nonceGroup_;
+ /**
+ * <code>optional uint64 nonceGroup = 2;</code>
+ */
+ public boolean hasNonceGroup() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>optional uint64 nonceGroup = 2;</code>
+ */
+ public long getNonceGroup() {
+ return nonceGroup_;
+ }
+
private void initFields() {
regionAction_ = java.util.Collections.emptyList();
+ nonceGroup_ = 0L;
}
private byte memoizedIsInitialized = -1;
public final boolean isInitialized() {
@@ -25621,6 +25834,9 @@ public final class ClientProtos {
for (int i = 0; i < regionAction_.size(); i++) {
output.writeMessage(1, regionAction_.get(i));
}
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ output.writeUInt64(2, nonceGroup_);
+ }
getUnknownFields().writeTo(output);
}
@@ -25634,6 +25850,10 @@ public final class ClientProtos {
size += com.google.protobuf.CodedOutputStream
.computeMessageSize(1, regionAction_.get(i));
}
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(2, nonceGroup_);
+ }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
@@ -25659,6 +25879,11 @@ public final class ClientProtos {
boolean result = true;
result = result && getRegionActionList()
.equals(other.getRegionActionList());
+ result = result && (hasNonceGroup() == other.hasNonceGroup());
+ if (hasNonceGroup()) {
+ result = result && (getNonceGroup()
+ == other.getNonceGroup());
+ }
result = result &&
getUnknownFields().equals(other.getUnknownFields());
return result;
@@ -25676,6 +25901,10 @@ public final class ClientProtos {
hash = (37 * hash) + REGIONACTION_FIELD_NUMBER;
hash = (53 * hash) + getRegionActionList().hashCode();
}
+ if (hasNonceGroup()) {
+ hash = (37 * hash) + NONCEGROUP_FIELD_NUMBER;
+ hash = (53 * hash) + hashLong(getNonceGroup());
+ }
hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash;
return hash;
@@ -25801,6 +26030,8 @@ public final class ClientProtos {
} else {
regionActionBuilder_.clear();
}
+ nonceGroup_ = 0L;
+ bitField0_ = (bitField0_ & ~0x00000002);
return this;
}
@@ -25828,6 +26059,7 @@ public final class ClientProtos {
public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest buildPartial() {
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest(this);
int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
if (regionActionBuilder_ == null) {
if (((bitField0_ & 0x00000001) == 0x00000001)) {
regionAction_ = java.util.Collections.unmodifiableList(regionAction_);
@@ -25837,6 +26069,11 @@ public final class ClientProtos {
} else {
result.regionAction_ = regionActionBuilder_.build();
}
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ result.nonceGroup_ = nonceGroup_;
+ result.bitField0_ = to_bitField0_;
onBuilt();
return result;
}
@@ -25878,6 +26115,9 @@ public final class ClientProtos {
}
}
}
+ if (other.hasNonceGroup()) {
+ setNonceGroup(other.getNonceGroup());
+ }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@@ -26151,6 +26391,39 @@ public final class ClientProtos {
return regionActionBuilder_;
}
+ // optional uint64 nonceGroup = 2;
+ private long nonceGroup_ ;
+ /**
+ * <code>optional uint64 nonceGroup = 2;</code>
+ */
+ public boolean hasNonceGroup() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional uint64 nonceGroup = 2;</code>
+ */
+ public long getNonceGroup() {
+ return nonceGroup_;
+ }
+ /**
+ * <code>optional uint64 nonceGroup = 2;</code>
+ */
+ public Builder setNonceGroup(long value) {
+ bitField0_ |= 0x00000002;
+ nonceGroup_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional uint64 nonceGroup = 2;</code>
+ */
+ public Builder clearNonceGroup() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ nonceGroup_ = 0L;
+ onChanged();
+ return this;
+ }
+
// @@protoc_insertion_point(builder_scope:MultiRequest)
}
@@ -27627,7 +27900,7 @@ public final class ClientProtos {
"t\"\200\001\n\tCondition\022\013\n\003row\030\001 \002(\014\022\016\n\006family\030\002" +
" \002(\014\022\021\n\tqualifier\030\003 \002(\014\022\"\n\014compare_type\030" +
"\004 \002(\0162\014.CompareType\022\037\n\ncomparator\030\005 \002(\0132" +
- "\013.Comparator\"\246\006\n\rMutationProto\022\013\n\003row\030\001 " +
+ "\013.Comparator\"\265\006\n\rMutationProto\022\013\n\003row\030\001 " +
"\001(\014\0220\n\013mutate_type\030\002 \001(\0162\033.MutationProto",
".MutationType\0220\n\014column_value\030\003 \003(\0132\032.Mu" +
"tationProto.ColumnValue\022\021\n\ttimestamp\030\004 \001" +
@@ -27635,74 +27908,76 @@ public final class ClientProtos {
"\ndurability\030\006 \001(\0162\031.MutationProto.Durabi" +
"lity:\013USE_DEFAULT\022\036\n\ntime_range\030\007 \001(\0132\n." +
"TimeRange\022\035\n\025associated_cell_count\030\010 \001(\005" +
- "\032\347\001\n\013ColumnValue\022\016\n\006family\030\001 \002(\014\022B\n\017qual" +
- "ifier_value\030\002 \003(\0132).MutationProto.Column" +
- "Value.QualifierValue\032\203\001\n\016QualifierValue\022" +
- "\021\n\tqualifier\030\001 \001(\014\022\r\n\005value\030\002 \001(\014\022\021\n\ttim",
- "estamp\030\003 \001(\004\022.\n\013delete_type\030\004 \001(\0162\031.Muta" +
- "tionProto.DeleteType\022\014\n\004tags\030\005 \001(\014\"W\n\nDu" +
- "rability\022\017\n\013USE_DEFAULT\020\000\022\014\n\010SKIP_WAL\020\001\022" +
- "\r\n\tASYNC_WAL\020\002\022\014\n\010SYNC_WAL\020\003\022\r\n\tFSYNC_WA" +
- "L\020\004\">\n\014MutationType\022\n\n\006APPEND\020\000\022\r\n\tINCRE" +
- "MENT\020\001\022\007\n\003PUT\020\002\022\n\n\006DELETE\020\003\"p\n\nDeleteTyp" +
- "e\022\026\n\022DELETE_ONE_VERSION\020\000\022\034\n\030DELETE_MULT" +
- "IPLE_VERSIONS\020\001\022\021\n\rDELETE_FAMILY\020\002\022\031\n\025DE" +
- "LETE_FAMILY_VERSION\020\003\"r\n\rMutateRequest\022 " +
- "\n\006region\030\001 \002(\0132\020.RegionSpecifier\022 \n\010muta",
- "tion\030\002 \002(\0132\016.MutationProto\022\035\n\tcondition\030" +
- "\003 \001(\0132\n.Condition\"<\n\016MutateResponse\022\027\n\006r" +
- "esult\030\001 \001(\0132\007.Result\022\021\n\tprocessed\030\002 \001(\010\"" +
- "\344\002\n\004Scan\022\027\n\006column\030\001 \003(\0132\007.Column\022!\n\tatt" +
- "ribute\030\002 \003(\0132\016.NameBytesPair\022\021\n\tstart_ro" +
- "w\030\003 \001(\014\022\020\n\010stop_row\030\004 \001(\014\022\027\n\006filter\030\005 \001(" +
- "\0132\007.Filter\022\036\n\ntime_range\030\006 \001(\0132\n.TimeRan" +
- "ge\022\027\n\014max_versions\030\007 \001(\r:\0011\022\032\n\014cache_blo" +
- "cks\030\010 \001(\010:\004true\022\022\n\nbatch_size\030\t \001(\r\022\027\n\017m" +
- "ax_result_size\030\n \001(\004\022\023\n\013store_limit\030\013 \001(",
- "\r\022\024\n\014store_offset\030\014 \001(\r\022&\n\036load_column_f" +
- "amilies_on_demand\030\r \001(\010\022\r\n\005small\030\016 \001(\010\"\236" +
- "\001\n\013ScanRequest\022 \n\006region\030\001 \001(\0132\020.RegionS" +
- "pecifier\022\023\n\004scan\030\002 \001(\0132\005.Scan\022\022\n\nscanner" +
- "_id\030\003 \001(\004\022\026\n\016number_of_rows\030\004 \001(\r\022\025\n\rclo" +
- "se_scanner\030\005 \001(\010\022\025\n\rnext_call_seq\030\006 \001(\004\"" +
- "y\n\014ScanResponse\022\030\n\020cells_per_result\030\001 \003(" +
- "\r\022\022\n\nscanner_id\030\002 \001(\004\022\024\n\014more_results\030\003 " +
- "\001(\010\022\013\n\003ttl\030\004 \001(\r\022\030\n\007results\030\005 \003(\0132\007.Resu" +
- "lt\"\263\001\n\024BulkLoadHFileRequest\022 \n\006region\030\001 ",
- "\002(\0132\020.RegionSpecifier\0225\n\013family_path\030\002 \003" +
- "(\0132 .BulkLoadHFileRequest.FamilyPath\022\026\n\016" +
- "assign_seq_num\030\003 \001(\010\032*\n\nFamilyPath\022\016\n\006fa" +
- "mily\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFil" +
- "eResponse\022\016\n\006loaded\030\001 \002(\010\"a\n\026Coprocessor" +
- "ServiceCall\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\031CoprocessorServiceRequest\022 \n\006reg" +
- "ion\030\001 \002(\0132\020.RegionSpecifier\022%\n\004call\030\002 \002(" +
- "\0132\027.CoprocessorServiceCall\"]\n\032Coprocesso",
- "rServiceResponse\022 \n\006region\030\001 \002(\0132\020.Regio" +
- "nSpecifier\022\035\n\005value\030\002 \002(\0132\016.NameBytesPai" +
- "r\"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.Region" +
- "Specifier\022\016\n\006atomic\030\002 \001(\010\022\027\n\006action\030\003 \003(" +
- "\0132\007.Action\"^\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\"f\n\022RegionActi" +
- "onResult\022-\n\021resultOrException\030\001 \003(\0132\022.Re",
- "sultOrException\022!\n\texception\030\002 \001(\0132\016.Nam" +
- "eBytesPair\"3\n\014MultiRequest\022#\n\014regionActi" +
- "on\030\001 \003(\0132\r.RegionAction\"@\n\rMultiResponse" +
- "\022/\n\022regionActionResult\030\001 \003(\0132\023.RegionAct" +
- "ionResult2\261\002\n\rClientService\022 \n\003Get\022\013.Get" +
- "Request\032\014.GetResponse\022)\n\006Mutate\022\016.Mutate" +
- "Request\032\017.MutateResponse\022#\n\004Scan\022\014.ScanR" +
- "equest\032\r.ScanResponse\022>\n\rBulkLoadHFile\022\025" +
- ".BulkLoadHFileRequest\032\026.BulkLoadHFileRes" +
- "ponse\022F\n\013ExecService\022\032.CoprocessorServic",
- "eRequest\032\033.CoprocessorServiceResponse\022&\n" +
- "\005Multi\022\r.MultiRequest\032\016.MultiResponseBB\n" +
- "*org.apache.hadoop.hbase.protobuf.genera" +
- "tedB\014ClientProtosH\001\210\001\001\240\001\001"
+ "\022\r\n\005nonce\030\t \001(\004\032\347\001\n\013ColumnValue\022\016\n\006famil" +
+ "y\030\001 \002(\014\022B\n\017qualifier_value\030\002 \003(\0132).Mutat" +
+ "ionProto.ColumnValue.QualifierValue\032\203\001\n\016" +
+ "QualifierValue\022\021\n\tqualifier\030\001 \001(\014\022\r\n\005val",
+ "ue\030\002 \001(\014\022\021\n\ttimestamp\030\003 \001(\004\022.\n\013delete_ty" +
+ "pe\030\004 \001(\0162\031.MutationProto.DeleteType\022\014\n\004t" +
+ "ags\030\005 \001(\014\"W\n\nDurability\022\017\n\013USE_DEFAULT\020\000" +
+ "\022\014\n\010SKIP_WAL\020\001\022\r\n\tASYNC_WAL\020\002\022\014\n\010SYNC_WA" +
+ "L\020\003\022\r\n\tFSYNC_WAL\020\004\">\n\014MutationType\022\n\n\006AP" +
+ "PEND\020\000\022\r\n\tINCREMENT\020\001\022\007\n\003PUT\020\002\022\n\n\006DELETE" +
+ "\020\003\"p\n\nDeleteType\022\026\n\022DELETE_ONE_VERSION\020\000" +
+ "\022\034\n\030DELETE_MULTIPLE_VERSIONS\020\001\022\021\n\rDELETE" +
+ "_FAMILY\020\002\022\031\n\025DELETE_FAMILY_VERSION\020\003\"\207\001\n" +
+ "\rMutateRequest\022 \n\006region\030\001 \002(\0132\020.RegionS",
+ "pecifier\022 \n\010mutation\030\002 \002(\0132\016.MutationPro" +
+ "to\022\035\n\tcondition\030\003 \001(\0132\n.Condition\022\023\n\013non" +
+ "ce_group\030\004 \001(\004\"<\n\016MutateResponse\022\027\n\006resu" +
+ "lt\030\001 \001(\0132\007.Result\022\021\n\tprocessed\030\002 \001(\010\"\344\002\n" +
+ "\004Scan\022\027\n\006column\030\001 \003(\0132\007.Column\022!\n\tattrib" +
+ "ute\030\002 \003(\0132\016.NameBytesPair\022\021\n\tstart_row\030\003" +
+ " \001(\014\022\020\n\010stop_row\030\004 \001(\014\022\027\n\006filter\030\005 \001(\0132\007" +
+ ".Filter\022\036\n\ntime_range\030\006 \001(\0132\n.TimeRange\022" +
+ "\027\n\014max_versions\030\007 \001(\r:\0011\022\032\n\014cache_blocks" +
+ "\030\010 \001(\010:\004true\022\022\n\nbatch_size\030\t \001(\r\022\027\n\017max_",
+ "result_size\030\n \001(\004\022\023\n\013store_limit\030\013 \001(\r\022\024" +
+ "\n\014store_offset\030\014 \001(\r\022&\n\036load_column_fami" +
+ "lies_on_demand\030\r \001(\010\022\r\n\005small\030\016 \001(\010\"\236\001\n\013" +
+ "ScanRequest\022 \n\006region\030\001 \001(\0132\020.RegionSpec" +
+ "ifier\022\023\n\004scan\030\002 \001(\0132\005.Scan\022\022\n\nscanner_id" +
+ "\030\003 \001(\004\022\026\n\016number_of_rows\030\004 \001(\r\022\025\n\rclose_" +
+ "scanner\030\005 \001(\010\022\025\n\rnext_call_seq\030\006 \001(\004\"y\n\014" +
+ "ScanResponse\022\030\n\020cells_per_result\030\001 \003(\r\022\022" +
+ "\n\nscanner_id\030\002 \001(\004\022\024\n\014more_results\030\003 \001(\010" +
+ "\022\013\n\003ttl\030\004 \001(\r\022\030\n\007results\030\005 \003(\0132\007.Result\"",
+ "\263\001\n\024BulkLoadHFileRequest\022 \n\006region\030\001 \002(\013" +
+ "2\020.RegionSpecifier\0225\n\013family_path\030\002 \003(\0132" +
+ " .BulkLoadHFileRequest.FamilyPath\022\026\n\016ass" +
+ "ign_seq_num\030\003 \001(\010\032*\n\nFamilyPath\022\016\n\006famil" +
+ "y\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFileRe" +
+ "sponse\022\016\n\006loaded\030\001 \002(\010\"a\n\026CoprocessorSer" +
+ "viceCall\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\031CoprocessorServiceRequest\022 \n\006region" +
+ "\030\001 \002(\0132\020.RegionSpecifier\022%\n\004call\030\002 \002(\0132\027",
+ ".CoprocessorServiceCall\"]\n\032CoprocessorSe" +
+ "rviceResponse\022 \n\006region\030\001 \002(\0132\020.RegionSp" +
+ "ecifier\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\014" +
+ "RegionAction\022 \n\006region\030\001 \002(\0132\020.RegionSpe" +
+ "cifier\022\016\n\006atomic\030\002 \001(\010\022\027\n\006action\030\003 \003(\0132\007" +
+ ".Action\"^\n\021ResultOrException\022\r\n\005index\030\001 " +
+ "\001(\r\022\027\n\006result\030\002 \001(\0132\007.Result\022!\n\texceptio" +
+ "n\030\003 \001(\0132\016.NameBytesPair\"f\n\022RegionActionR",
+ "esult\022-\n\021resultOrException\030\001 \003(\0132\022.Resul" +
+ "tOrException\022!\n\texception\030\002 \001(\0132\016.NameBy" +
+ "tesPair\"G\n\014MultiRequest\022#\n\014regionAction\030" +
+ "\001 \003(\0132\r.RegionAction\022\022\n\nnonceGroup\030\002 \001(\004" +
+ "\"@\n\rMultiResponse\022/\n\022regionActionResult\030" +
+ "\001 \003(\0132\023.RegionActionResult2\261\002\n\rClientSer" +
+ "vice\022 \n\003Get\022\013.GetRequest\032\014.GetResponse\022)" +
+ "\n\006Mutate\022\016.MutateRequest\032\017.MutateRespons" +
+ "e\022#\n\004Scan\022\014.ScanRequest\032\r.ScanResponse\022>" +
+ "\n\rBulkLoadHFile\022\025.BulkLoadHFileRequest\032\026",
+ ".BulkLoadHFileResponse\022F\n\013ExecService\022\032." +
+ "CoprocessorServiceRequest\032\033.CoprocessorS" +
+ "erviceResponse\022&\n\005Multi\022\r.MultiRequest\032\016" +
+ ".MultiResponseBB\n*org.apache.hadoop.hbas" +
+ "e.protobuf.generatedB\014ClientProtosH\001\210\001\001\240" +
+ "\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -27750,7 +28025,7 @@ public final class ClientProtos {
internal_static_MutationProto_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_MutationProto_descriptor,
- new java.lang.String[] { "Row", "MutateType", "ColumnValue", "Timestamp", "Attribute", "Durability", "TimeRange", "AssociatedCellCount", });
+ new java.lang.String[] { "Row", "MutateType", "ColumnValue", "Timestamp", "Attribute", "Durability", "TimeRange", "AssociatedCellCount", "Nonce", });
internal_static_MutationProto_ColumnValue_descriptor =
internal_static_MutationProto_descriptor.getNestedTypes().get(0);
internal_static_MutationProto_ColumnValue_fieldAccessorTable = new
@@ -27768,7 +28043,7 @@ public final class ClientProtos {
internal_static_MutateRequest_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_MutateRequest_descriptor,
- new java.lang.String[] { "Region", "Mutation", "Condition", });
+ new java.lang.String[] { "Region", "Mutation", "Condition", "NonceGroup", });
internal_static_MutateResponse_descriptor =
getDescriptor().getMessageTypes().get(8);
internal_static_MutateResponse_fieldAccessorTable = new
@@ -27858,7 +28133,7 @@ public final class ClientProtos {
internal_static_MultiRequest_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_MultiRequest_descriptor,
- new java.lang.String[] { "RegionAction", });
+ new java.lang.String[] { "RegionAction", "NonceGroup", });
internal_static_MultiResponse_descriptor =
getDescriptor().getMessageTypes().get(22);
internal_static_MultiResponse_fieldAccessorTable = new