HBASE-3787 Increment is non-idempotent but client retries RPC
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1542168 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
db4f10c208
commit
6119df9ccb
|
@ -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 Comparable<R> {
|
|||
// 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 Comparable<R> {
|
|||
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 Comparable<R> {
|
|||
Action<?> other = (Action<?>) obj;
|
||||
return compareTo(other) == 0;
|
||||
}
|
||||
|
||||
public long getNonce() {
|
||||
return nonce;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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];
|
||||
}
|
||||
}
|
|
@ -515,4 +515,9 @@ public interface HConnection extends Abortable, Closeable {
|
|||
* @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();
|
||||
}
|
|
@ -124,6 +124,7 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
|||
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
|
||||
|
@ -229,6 +239,20 @@ public class HConnectionManager {
|
|||
super();
|
||||
}
|
||||
|
||||
/**
|
||||
* @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
|
||||
|
@ -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
|
||||
|
|
|
@ -986,12 +986,15 @@ public class HTable implements HTableInterface {
|
|||
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 HTableInterface {
|
|||
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 HTableInterface {
|
|||
"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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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 RegionServerCallable<MultiResponse> {
|
|||
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 RegionServerCallable<MultiResponse> {
|
|||
}
|
||||
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);
|
||||
|
|
|
@ -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();
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
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, 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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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.
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -711,6 +711,26 @@ public final class MultiRowMutationProtos {
|
|||
*/
|
||||
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getMutationRequestOrBuilder(
|
||||
int index);
|
||||
|
||||
// optional uint64 nonce_group = 2;
|
||||
/**
|
||||
* <code>optional uint64 nonce_group = 2;</code>
|
||||
*/
|
||||
boolean hasNonceGroup();
|
||||
/**
|
||||
* <code>optional uint64 nonce_group = 2;</code>
|
||||
*/
|
||||
long getNonceGroup();
|
||||
|
||||
// optional uint64 nonce = 3;
|
||||
/**
|
||||
* <code>optional uint64 nonce = 3;</code>
|
||||
*/
|
||||
boolean hasNonce();
|
||||
/**
|
||||
* <code>optional uint64 nonce = 3;</code>
|
||||
*/
|
||||
long getNonce();
|
||||
}
|
||||
/**
|
||||
* Protobuf type {@code MutateRowsRequest}
|
||||
|
@ -771,6 +791,16 @@ public final class MultiRowMutationProtos {
|
|||
mutationRequest_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.PARSER, extensionRegistry));
|
||||
break;
|
||||
}
|
||||
case 16: {
|
||||
bitField0_ |= 0x00000001;
|
||||
nonceGroup_ = input.readUInt64();
|
||||
break;
|
||||
}
|
||||
case 24: {
|
||||
bitField0_ |= 0x00000002;
|
||||
nonce_ = input.readUInt64();
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
|
||||
|
@ -813,6 +843,7 @@ public final class MultiRowMutationProtos {
|
|||
return PARSER;
|
||||
}
|
||||
|
||||
private int bitField0_;
|
||||
// repeated .MutationProto mutation_request = 1;
|
||||
public static final int MUTATION_REQUEST_FIELD_NUMBER = 1;
|
||||
private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto> mutationRequest_;
|
||||
|
@ -849,8 +880,42 @@ public final class MultiRowMutationProtos {
|
|||
return mutationRequest_.get(index);
|
||||
}
|
||||
|
||||
// optional uint64 nonce_group = 2;
|
||||
public static final int NONCE_GROUP_FIELD_NUMBER = 2;
|
||||
private long nonceGroup_;
|
||||
/**
|
||||
* <code>optional uint64 nonce_group = 2;</code>
|
||||
*/
|
||||
public boolean hasNonceGroup() {
|
||||
return ((bitField0_ & 0x00000001) == 0x00000001);
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonce_group = 2;</code>
|
||||
*/
|
||||
public long getNonceGroup() {
|
||||
return nonceGroup_;
|
||||
}
|
||||
|
||||
// optional uint64 nonce = 3;
|
||||
public static final int NONCE_FIELD_NUMBER = 3;
|
||||
private long nonce_;
|
||||
/**
|
||||
* <code>optional uint64 nonce = 3;</code>
|
||||
*/
|
||||
public boolean hasNonce() {
|
||||
return ((bitField0_ & 0x00000002) == 0x00000002);
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonce = 3;</code>
|
||||
*/
|
||||
public long getNonce() {
|
||||
return nonce_;
|
||||
}
|
||||
|
||||
private void initFields() {
|
||||
mutationRequest_ = java.util.Collections.emptyList();
|
||||
nonceGroup_ = 0L;
|
||||
nonce_ = 0L;
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
|
@ -873,6 +938,12 @@ public final class MultiRowMutationProtos {
|
|||
for (int i = 0; i < mutationRequest_.size(); i++) {
|
||||
output.writeMessage(1, mutationRequest_.get(i));
|
||||
}
|
||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
output.writeUInt64(2, nonceGroup_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
output.writeUInt64(3, nonce_);
|
||||
}
|
||||
getUnknownFields().writeTo(output);
|
||||
}
|
||||
|
||||
|
@ -886,6 +957,14 @@ public final class MultiRowMutationProtos {
|
|||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeMessageSize(1, mutationRequest_.get(i));
|
||||
}
|
||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeUInt64Size(2, nonceGroup_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeUInt64Size(3, nonce_);
|
||||
}
|
||||
size += getUnknownFields().getSerializedSize();
|
||||
memoizedSerializedSize = size;
|
||||
return size;
|
||||
|
@ -911,6 +990,16 @@ public final class MultiRowMutationProtos {
|
|||
boolean result = true;
|
||||
result = result && getMutationRequestList()
|
||||
.equals(other.getMutationRequestList());
|
||||
result = result && (hasNonceGroup() == other.hasNonceGroup());
|
||||
if (hasNonceGroup()) {
|
||||
result = result && (getNonceGroup()
|
||||
== other.getNonceGroup());
|
||||
}
|
||||
result = result && (hasNonce() == other.hasNonce());
|
||||
if (hasNonce()) {
|
||||
result = result && (getNonce()
|
||||
== other.getNonce());
|
||||
}
|
||||
result = result &&
|
||||
getUnknownFields().equals(other.getUnknownFields());
|
||||
return result;
|
||||
|
@ -928,6 +1017,14 @@ public final class MultiRowMutationProtos {
|
|||
hash = (37 * hash) + MUTATION_REQUEST_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getMutationRequestList().hashCode();
|
||||
}
|
||||
if (hasNonceGroup()) {
|
||||
hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
|
||||
hash = (53 * hash) + hashLong(getNonceGroup());
|
||||
}
|
||||
if (hasNonce()) {
|
||||
hash = (37 * hash) + NONCE_FIELD_NUMBER;
|
||||
hash = (53 * hash) + hashLong(getNonce());
|
||||
}
|
||||
hash = (29 * hash) + getUnknownFields().hashCode();
|
||||
memoizedHashCode = hash;
|
||||
return hash;
|
||||
|
@ -1044,6 +1141,10 @@ public final class MultiRowMutationProtos {
|
|||
} else {
|
||||
mutationRequestBuilder_.clear();
|
||||
}
|
||||
nonceGroup_ = 0L;
|
||||
bitField0_ = (bitField0_ & ~0x00000002);
|
||||
nonce_ = 0L;
|
||||
bitField0_ = (bitField0_ & ~0x00000004);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -1071,6 +1172,7 @@ public final class MultiRowMutationProtos {
|
|||
public org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest buildPartial() {
|
||||
org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest result = new org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest(this);
|
||||
int from_bitField0_ = bitField0_;
|
||||
int to_bitField0_ = 0;
|
||||
if (mutationRequestBuilder_ == null) {
|
||||
if (((bitField0_ & 0x00000001) == 0x00000001)) {
|
||||
mutationRequest_ = java.util.Collections.unmodifiableList(mutationRequest_);
|
||||
|
@ -1080,6 +1182,15 @@ public final class MultiRowMutationProtos {
|
|||
} else {
|
||||
result.mutationRequest_ = mutationRequestBuilder_.build();
|
||||
}
|
||||
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
|
||||
to_bitField0_ |= 0x00000001;
|
||||
}
|
||||
result.nonceGroup_ = nonceGroup_;
|
||||
if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
|
||||
to_bitField0_ |= 0x00000002;
|
||||
}
|
||||
result.nonce_ = nonce_;
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
}
|
||||
|
@ -1121,6 +1232,12 @@ public final class MultiRowMutationProtos {
|
|||
}
|
||||
}
|
||||
}
|
||||
if (other.hasNonceGroup()) {
|
||||
setNonceGroup(other.getNonceGroup());
|
||||
}
|
||||
if (other.hasNonce()) {
|
||||
setNonce(other.getNonce());
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
}
|
||||
|
@ -1394,6 +1511,72 @@ public final class MultiRowMutationProtos {
|
|||
return mutationRequestBuilder_;
|
||||
}
|
||||
|
||||
// optional uint64 nonce_group = 2;
|
||||
private long nonceGroup_ ;
|
||||
/**
|
||||
* <code>optional uint64 nonce_group = 2;</code>
|
||||
*/
|
||||
public boolean hasNonceGroup() {
|
||||
return ((bitField0_ & 0x00000002) == 0x00000002);
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonce_group = 2;</code>
|
||||
*/
|
||||
public long getNonceGroup() {
|
||||
return nonceGroup_;
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonce_group = 2;</code>
|
||||
*/
|
||||
public Builder setNonceGroup(long value) {
|
||||
bitField0_ |= 0x00000002;
|
||||
nonceGroup_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonce_group = 2;</code>
|
||||
*/
|
||||
public Builder clearNonceGroup() {
|
||||
bitField0_ = (bitField0_ & ~0x00000002);
|
||||
nonceGroup_ = 0L;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
// optional uint64 nonce = 3;
|
||||
private long nonce_ ;
|
||||
/**
|
||||
* <code>optional uint64 nonce = 3;</code>
|
||||
*/
|
||||
public boolean hasNonce() {
|
||||
return ((bitField0_ & 0x00000004) == 0x00000004);
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonce = 3;</code>
|
||||
*/
|
||||
public long getNonce() {
|
||||
return nonce_;
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonce = 3;</code>
|
||||
*/
|
||||
public Builder setNonce(long value) {
|
||||
bitField0_ |= 0x00000004;
|
||||
nonce_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonce = 3;</code>
|
||||
*/
|
||||
public Builder clearNonce() {
|
||||
bitField0_ = (bitField0_ & ~0x00000004);
|
||||
nonce_ = 0L;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(builder_scope:MutateRowsRequest)
|
||||
}
|
||||
|
||||
|
@ -2006,13 +2189,14 @@ public final class MultiRowMutationProtos {
|
|||
java.lang.String[] descriptorData = {
|
||||
"\n\026MultiRowMutation.proto\032\014Client.proto\"\"" +
|
||||
"\n MultiRowMutationProcessorRequest\"#\n!Mu" +
|
||||
"ltiRowMutationProcessorResponse\"=\n\021Mutat" +
|
||||
"ltiRowMutationProcessorResponse\"a\n\021Mutat" +
|
||||
"eRowsRequest\022(\n\020mutation_request\030\001 \003(\0132\016" +
|
||||
".MutationProto\"\024\n\022MutateRowsResponse2P\n\027" +
|
||||
"MultiRowMutationService\0225\n\nMutateRows\022\022." +
|
||||
"MutateRowsRequest\032\023.MutateRowsResponseBL" +
|
||||
"\n*org.apache.hadoop.hbase.protobuf.gener" +
|
||||
"atedB\026MultiRowMutationProtosH\001\210\001\001\240\001\001"
|
||||
".MutationProto\022\023\n\013nonce_group\030\002 \001(\004\022\r\n\005n" +
|
||||
"once\030\003 \001(\004\"\024\n\022MutateRowsResponse2P\n\027Mult" +
|
||||
"iRowMutationService\0225\n\nMutateRows\022\022.Muta" +
|
||||
"teRowsRequest\032\023.MutateRowsResponseBL\n*or" +
|
||||
"g.apache.hadoop.hbase.protobuf.generated" +
|
||||
"B\026MultiRowMutationProtosH\001\210\001\001\240\001\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
@ -2036,7 +2220,7 @@ public final class MultiRowMutationProtos {
|
|||
internal_static_MutateRowsRequest_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_MutateRowsRequest_descriptor,
|
||||
new java.lang.String[] { "MutationRequest", });
|
||||
new java.lang.String[] { "MutationRequest", "NonceGroup", "Nonce", });
|
||||
internal_static_MutateRowsResponse_descriptor =
|
||||
getDescriptor().getMessageTypes().get(3);
|
||||
internal_static_MutateRowsResponse_fieldAccessorTable = new
|
||||
|
|
|
@ -50,6 +50,26 @@ public final class RowProcessorProtos {
|
|||
* <code>optional bytes row_processor_initializer_message = 3;</code>
|
||||
*/
|
||||
com.google.protobuf.ByteString getRowProcessorInitializerMessage();
|
||||
|
||||
// optional uint64 nonce_group = 4;
|
||||
/**
|
||||
* <code>optional uint64 nonce_group = 4;</code>
|
||||
*/
|
||||
boolean hasNonceGroup();
|
||||
/**
|
||||
* <code>optional uint64 nonce_group = 4;</code>
|
||||
*/
|
||||
long getNonceGroup();
|
||||
|
||||
// optional uint64 nonce = 5;
|
||||
/**
|
||||
* <code>optional uint64 nonce = 5;</code>
|
||||
*/
|
||||
boolean hasNonce();
|
||||
/**
|
||||
* <code>optional uint64 nonce = 5;</code>
|
||||
*/
|
||||
long getNonce();
|
||||
}
|
||||
/**
|
||||
* Protobuf type {@code ProcessRequest}
|
||||
|
@ -117,6 +137,16 @@ public final class RowProcessorProtos {
|
|||
rowProcessorInitializerMessage_ = input.readBytes();
|
||||
break;
|
||||
}
|
||||
case 32: {
|
||||
bitField0_ |= 0x00000008;
|
||||
nonceGroup_ = input.readUInt64();
|
||||
break;
|
||||
}
|
||||
case 40: {
|
||||
bitField0_ |= 0x00000010;
|
||||
nonce_ = input.readUInt64();
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
|
||||
|
@ -259,10 +289,44 @@ public final class RowProcessorProtos {
|
|||
return rowProcessorInitializerMessage_;
|
||||
}
|
||||
|
||||
// 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_;
|
||||
}
|
||||
|
||||
// optional uint64 nonce = 5;
|
||||
public static final int NONCE_FIELD_NUMBER = 5;
|
||||
private long nonce_;
|
||||
/**
|
||||
* <code>optional uint64 nonce = 5;</code>
|
||||
*/
|
||||
public boolean hasNonce() {
|
||||
return ((bitField0_ & 0x00000010) == 0x00000010);
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonce = 5;</code>
|
||||
*/
|
||||
public long getNonce() {
|
||||
return nonce_;
|
||||
}
|
||||
|
||||
private void initFields() {
|
||||
rowProcessorClassName_ = "";
|
||||
rowProcessorInitializerMessageName_ = "";
|
||||
rowProcessorInitializerMessage_ = com.google.protobuf.ByteString.EMPTY;
|
||||
nonceGroup_ = 0L;
|
||||
nonce_ = 0L;
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
|
@ -289,6 +353,12 @@ public final class RowProcessorProtos {
|
|||
if (((bitField0_ & 0x00000004) == 0x00000004)) {
|
||||
output.writeBytes(3, rowProcessorInitializerMessage_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000008) == 0x00000008)) {
|
||||
output.writeUInt64(4, nonceGroup_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000010) == 0x00000010)) {
|
||||
output.writeUInt64(5, nonce_);
|
||||
}
|
||||
getUnknownFields().writeTo(output);
|
||||
}
|
||||
|
||||
|
@ -310,6 +380,14 @@ public final class RowProcessorProtos {
|
|||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeBytesSize(3, rowProcessorInitializerMessage_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000008) == 0x00000008)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeUInt64Size(4, nonceGroup_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000010) == 0x00000010)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeUInt64Size(5, nonce_);
|
||||
}
|
||||
size += getUnknownFields().getSerializedSize();
|
||||
memoizedSerializedSize = size;
|
||||
return size;
|
||||
|
@ -348,6 +426,16 @@ public final class RowProcessorProtos {
|
|||
result = result && getRowProcessorInitializerMessage()
|
||||
.equals(other.getRowProcessorInitializerMessage());
|
||||
}
|
||||
result = result && (hasNonceGroup() == other.hasNonceGroup());
|
||||
if (hasNonceGroup()) {
|
||||
result = result && (getNonceGroup()
|
||||
== other.getNonceGroup());
|
||||
}
|
||||
result = result && (hasNonce() == other.hasNonce());
|
||||
if (hasNonce()) {
|
||||
result = result && (getNonce()
|
||||
== other.getNonce());
|
||||
}
|
||||
result = result &&
|
||||
getUnknownFields().equals(other.getUnknownFields());
|
||||
return result;
|
||||
|
@ -373,6 +461,14 @@ public final class RowProcessorProtos {
|
|||
hash = (37 * hash) + ROW_PROCESSOR_INITIALIZER_MESSAGE_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getRowProcessorInitializerMessage().hashCode();
|
||||
}
|
||||
if (hasNonceGroup()) {
|
||||
hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
|
||||
hash = (53 * hash) + hashLong(getNonceGroup());
|
||||
}
|
||||
if (hasNonce()) {
|
||||
hash = (37 * hash) + NONCE_FIELD_NUMBER;
|
||||
hash = (53 * hash) + hashLong(getNonce());
|
||||
}
|
||||
hash = (29 * hash) + getUnknownFields().hashCode();
|
||||
memoizedHashCode = hash;
|
||||
return hash;
|
||||
|
@ -488,6 +584,10 @@ public final class RowProcessorProtos {
|
|||
bitField0_ = (bitField0_ & ~0x00000002);
|
||||
rowProcessorInitializerMessage_ = com.google.protobuf.ByteString.EMPTY;
|
||||
bitField0_ = (bitField0_ & ~0x00000004);
|
||||
nonceGroup_ = 0L;
|
||||
bitField0_ = (bitField0_ & ~0x00000008);
|
||||
nonce_ = 0L;
|
||||
bitField0_ = (bitField0_ & ~0x00000010);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -528,6 +628,14 @@ public final class RowProcessorProtos {
|
|||
to_bitField0_ |= 0x00000004;
|
||||
}
|
||||
result.rowProcessorInitializerMessage_ = rowProcessorInitializerMessage_;
|
||||
if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
|
||||
to_bitField0_ |= 0x00000008;
|
||||
}
|
||||
result.nonceGroup_ = nonceGroup_;
|
||||
if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
|
||||
to_bitField0_ |= 0x00000010;
|
||||
}
|
||||
result.nonce_ = nonce_;
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
|
@ -557,6 +665,12 @@ public final class RowProcessorProtos {
|
|||
if (other.hasRowProcessorInitializerMessage()) {
|
||||
setRowProcessorInitializerMessage(other.getRowProcessorInitializerMessage());
|
||||
}
|
||||
if (other.hasNonceGroup()) {
|
||||
setNonceGroup(other.getNonceGroup());
|
||||
}
|
||||
if (other.hasNonce()) {
|
||||
setNonce(other.getNonce());
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
}
|
||||
|
@ -772,6 +886,72 @@ public final class RowProcessorProtos {
|
|||
return this;
|
||||
}
|
||||
|
||||
// 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;
|
||||
}
|
||||
|
||||
// optional uint64 nonce = 5;
|
||||
private long nonce_ ;
|
||||
/**
|
||||
* <code>optional uint64 nonce = 5;</code>
|
||||
*/
|
||||
public boolean hasNonce() {
|
||||
return ((bitField0_ & 0x00000010) == 0x00000010);
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonce = 5;</code>
|
||||
*/
|
||||
public long getNonce() {
|
||||
return nonce_;
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonce = 5;</code>
|
||||
*/
|
||||
public Builder setNonce(long value) {
|
||||
bitField0_ |= 0x00000010;
|
||||
nonce_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonce = 5;</code>
|
||||
*/
|
||||
public Builder clearNonce() {
|
||||
bitField0_ = (bitField0_ & ~0x00000010);
|
||||
nonce_ = 0L;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(builder_scope:ProcessRequest)
|
||||
}
|
||||
|
||||
|
@ -1479,15 +1659,16 @@ public final class RowProcessorProtos {
|
|||
descriptor;
|
||||
static {
|
||||
java.lang.String[] descriptorData = {
|
||||
"\n\022RowProcessor.proto\"\215\001\n\016ProcessRequest\022" +
|
||||
"\n\022RowProcessor.proto\"\261\001\n\016ProcessRequest\022" +
|
||||
" \n\030row_processor_class_name\030\001 \002(\t\022.\n&row" +
|
||||
"_processor_initializer_message_name\030\002 \001(" +
|
||||
"\t\022)\n!row_processor_initializer_message\030\003" +
|
||||
" \001(\014\"/\n\017ProcessResponse\022\034\n\024row_processor" +
|
||||
"_result\030\001 \002(\0142C\n\023RowProcessorService\022,\n\007" +
|
||||
"Process\022\017.ProcessRequest\032\020.ProcessRespon" +
|
||||
"seBH\n*org.apache.hadoop.hbase.protobuf.g" +
|
||||
"eneratedB\022RowProcessorProtosH\001\210\001\001\240\001\001"
|
||||
" \001(\014\022\023\n\013nonce_group\030\004 \001(\004\022\r\n\005nonce\030\005 \001(\004" +
|
||||
"\"/\n\017ProcessResponse\022\034\n\024row_processor_res" +
|
||||
"ult\030\001 \002(\0142C\n\023RowProcessorService\022,\n\007Proc" +
|
||||
"ess\022\017.ProcessRequest\032\020.ProcessResponseBH" +
|
||||
"\n*org.apache.hadoop.hbase.protobuf.gener" +
|
||||
"atedB\022RowProcessorProtosH\001\210\001\001\240\001\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
@ -1499,7 +1680,7 @@ public final class RowProcessorProtos {
|
|||
internal_static_ProcessRequest_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_ProcessRequest_descriptor,
|
||||
new java.lang.String[] { "RowProcessorClassName", "RowProcessorInitializerMessageName", "RowProcessorInitializerMessage", });
|
||||
new java.lang.String[] { "RowProcessorClassName", "RowProcessorInitializerMessageName", "RowProcessorInitializerMessage", "NonceGroup", "Nonce", });
|
||||
internal_static_ProcessResponse_descriptor =
|
||||
getDescriptor().getMessageTypes().get(1);
|
||||
internal_static_ProcessResponse_fieldAccessorTable = new
|
||||
|
|
|
@ -694,6 +694,26 @@ public final class WALProtos {
|
|||
*/
|
||||
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.UUIDOrBuilder getClusterIdsOrBuilder(
|
||||
int index);
|
||||
|
||||
// optional uint64 nonceGroup = 9;
|
||||
/**
|
||||
* <code>optional uint64 nonceGroup = 9;</code>
|
||||
*/
|
||||
boolean hasNonceGroup();
|
||||
/**
|
||||
* <code>optional uint64 nonceGroup = 9;</code>
|
||||
*/
|
||||
long getNonceGroup();
|
||||
|
||||
// optional uint64 nonce = 10;
|
||||
/**
|
||||
* <code>optional uint64 nonce = 10;</code>
|
||||
*/
|
||||
boolean hasNonce();
|
||||
/**
|
||||
* <code>optional uint64 nonce = 10;</code>
|
||||
*/
|
||||
long getNonce();
|
||||
}
|
||||
/**
|
||||
* Protobuf type {@code WALKey}
|
||||
|
@ -804,6 +824,16 @@ public final class WALProtos {
|
|||
clusterIds_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.UUID.PARSER, extensionRegistry));
|
||||
break;
|
||||
}
|
||||
case 72: {
|
||||
bitField0_ |= 0x00000040;
|
||||
nonceGroup_ = input.readUInt64();
|
||||
break;
|
||||
}
|
||||
case 80: {
|
||||
bitField0_ |= 0x00000080;
|
||||
nonce_ = input.readUInt64();
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (com.google.protobuf.InvalidProtocolBufferException e) {
|
||||
|
@ -1078,6 +1108,38 @@ public final class WALProtos {
|
|||
return clusterIds_.get(index);
|
||||
}
|
||||
|
||||
// optional uint64 nonceGroup = 9;
|
||||
public static final int NONCEGROUP_FIELD_NUMBER = 9;
|
||||
private long nonceGroup_;
|
||||
/**
|
||||
* <code>optional uint64 nonceGroup = 9;</code>
|
||||
*/
|
||||
public boolean hasNonceGroup() {
|
||||
return ((bitField0_ & 0x00000040) == 0x00000040);
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonceGroup = 9;</code>
|
||||
*/
|
||||
public long getNonceGroup() {
|
||||
return nonceGroup_;
|
||||
}
|
||||
|
||||
// optional uint64 nonce = 10;
|
||||
public static final int NONCE_FIELD_NUMBER = 10;
|
||||
private long nonce_;
|
||||
/**
|
||||
* <code>optional uint64 nonce = 10;</code>
|
||||
*/
|
||||
public boolean hasNonce() {
|
||||
return ((bitField0_ & 0x00000080) == 0x00000080);
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonce = 10;</code>
|
||||
*/
|
||||
public long getNonce() {
|
||||
return nonce_;
|
||||
}
|
||||
|
||||
private void initFields() {
|
||||
encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
|
||||
tableName_ = com.google.protobuf.ByteString.EMPTY;
|
||||
|
@ -1087,6 +1149,8 @@ public final class WALProtos {
|
|||
scopes_ = java.util.Collections.emptyList();
|
||||
followingKvCount_ = 0;
|
||||
clusterIds_ = java.util.Collections.emptyList();
|
||||
nonceGroup_ = 0L;
|
||||
nonce_ = 0L;
|
||||
}
|
||||
private byte memoizedIsInitialized = -1;
|
||||
public final boolean isInitialized() {
|
||||
|
@ -1158,6 +1222,12 @@ public final class WALProtos {
|
|||
for (int i = 0; i < clusterIds_.size(); i++) {
|
||||
output.writeMessage(8, clusterIds_.get(i));
|
||||
}
|
||||
if (((bitField0_ & 0x00000040) == 0x00000040)) {
|
||||
output.writeUInt64(9, nonceGroup_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000080) == 0x00000080)) {
|
||||
output.writeUInt64(10, nonce_);
|
||||
}
|
||||
getUnknownFields().writeTo(output);
|
||||
}
|
||||
|
||||
|
@ -1199,6 +1269,14 @@ public final class WALProtos {
|
|||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeMessageSize(8, clusterIds_.get(i));
|
||||
}
|
||||
if (((bitField0_ & 0x00000040) == 0x00000040)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeUInt64Size(9, nonceGroup_);
|
||||
}
|
||||
if (((bitField0_ & 0x00000080) == 0x00000080)) {
|
||||
size += com.google.protobuf.CodedOutputStream
|
||||
.computeUInt64Size(10, nonce_);
|
||||
}
|
||||
size += getUnknownFields().getSerializedSize();
|
||||
memoizedSerializedSize = size;
|
||||
return size;
|
||||
|
@ -1256,6 +1334,16 @@ public final class WALProtos {
|
|||
}
|
||||
result = result && getClusterIdsList()
|
||||
.equals(other.getClusterIdsList());
|
||||
result = result && (hasNonceGroup() == other.hasNonceGroup());
|
||||
if (hasNonceGroup()) {
|
||||
result = result && (getNonceGroup()
|
||||
== other.getNonceGroup());
|
||||
}
|
||||
result = result && (hasNonce() == other.hasNonce());
|
||||
if (hasNonce()) {
|
||||
result = result && (getNonce()
|
||||
== other.getNonce());
|
||||
}
|
||||
result = result &&
|
||||
getUnknownFields().equals(other.getUnknownFields());
|
||||
return result;
|
||||
|
@ -1301,6 +1389,14 @@ public final class WALProtos {
|
|||
hash = (37 * hash) + CLUSTER_IDS_FIELD_NUMBER;
|
||||
hash = (53 * hash) + getClusterIdsList().hashCode();
|
||||
}
|
||||
if (hasNonceGroup()) {
|
||||
hash = (37 * hash) + NONCEGROUP_FIELD_NUMBER;
|
||||
hash = (53 * hash) + hashLong(getNonceGroup());
|
||||
}
|
||||
if (hasNonce()) {
|
||||
hash = (37 * hash) + NONCE_FIELD_NUMBER;
|
||||
hash = (53 * hash) + hashLong(getNonce());
|
||||
}
|
||||
hash = (29 * hash) + getUnknownFields().hashCode();
|
||||
memoizedHashCode = hash;
|
||||
return hash;
|
||||
|
@ -1445,6 +1541,10 @@ public final class WALProtos {
|
|||
} else {
|
||||
clusterIdsBuilder_.clear();
|
||||
}
|
||||
nonceGroup_ = 0L;
|
||||
bitField0_ = (bitField0_ & ~0x00000100);
|
||||
nonce_ = 0L;
|
||||
bitField0_ = (bitField0_ & ~0x00000200);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -1519,6 +1619,14 @@ public final class WALProtos {
|
|||
} else {
|
||||
result.clusterIds_ = clusterIdsBuilder_.build();
|
||||
}
|
||||
if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
|
||||
to_bitField0_ |= 0x00000040;
|
||||
}
|
||||
result.nonceGroup_ = nonceGroup_;
|
||||
if (((from_bitField0_ & 0x00000200) == 0x00000200)) {
|
||||
to_bitField0_ |= 0x00000080;
|
||||
}
|
||||
result.nonce_ = nonce_;
|
||||
result.bitField0_ = to_bitField0_;
|
||||
onBuilt();
|
||||
return result;
|
||||
|
@ -1605,6 +1713,12 @@ public final class WALProtos {
|
|||
}
|
||||
}
|
||||
}
|
||||
if (other.hasNonceGroup()) {
|
||||
setNonceGroup(other.getNonceGroup());
|
||||
}
|
||||
if (other.hasNonce()) {
|
||||
setNonce(other.getNonce());
|
||||
}
|
||||
this.mergeUnknownFields(other.getUnknownFields());
|
||||
return this;
|
||||
}
|
||||
|
@ -2614,6 +2728,72 @@ public final class WALProtos {
|
|||
return clusterIdsBuilder_;
|
||||
}
|
||||
|
||||
// optional uint64 nonceGroup = 9;
|
||||
private long nonceGroup_ ;
|
||||
/**
|
||||
* <code>optional uint64 nonceGroup = 9;</code>
|
||||
*/
|
||||
public boolean hasNonceGroup() {
|
||||
return ((bitField0_ & 0x00000100) == 0x00000100);
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonceGroup = 9;</code>
|
||||
*/
|
||||
public long getNonceGroup() {
|
||||
return nonceGroup_;
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonceGroup = 9;</code>
|
||||
*/
|
||||
public Builder setNonceGroup(long value) {
|
||||
bitField0_ |= 0x00000100;
|
||||
nonceGroup_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonceGroup = 9;</code>
|
||||
*/
|
||||
public Builder clearNonceGroup() {
|
||||
bitField0_ = (bitField0_ & ~0x00000100);
|
||||
nonceGroup_ = 0L;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
// optional uint64 nonce = 10;
|
||||
private long nonce_ ;
|
||||
/**
|
||||
* <code>optional uint64 nonce = 10;</code>
|
||||
*/
|
||||
public boolean hasNonce() {
|
||||
return ((bitField0_ & 0x00000200) == 0x00000200);
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonce = 10;</code>
|
||||
*/
|
||||
public long getNonce() {
|
||||
return nonce_;
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonce = 10;</code>
|
||||
*/
|
||||
public Builder setNonce(long value) {
|
||||
bitField0_ |= 0x00000200;
|
||||
nonce_ = value;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* <code>optional uint64 nonce = 10;</code>
|
||||
*/
|
||||
public Builder clearNonce() {
|
||||
bitField0_ = (bitField0_ & ~0x00000200);
|
||||
nonce_ = 0L;
|
||||
onChanged();
|
||||
return this;
|
||||
}
|
||||
|
||||
// @@protoc_insertion_point(builder_scope:WALKey)
|
||||
}
|
||||
|
||||
|
@ -4812,22 +4992,23 @@ public final class WALProtos {
|
|||
static {
|
||||
java.lang.String[] descriptorData = {
|
||||
"\n\tWAL.proto\032\013HBase.proto\"$\n\tWALHeader\022\027\n" +
|
||||
"\017has_compression\030\001 \001(\010\"\337\001\n\006WALKey\022\033\n\023enc" +
|
||||
"\017has_compression\030\001 \001(\010\"\202\002\n\006WALKey\022\033\n\023enc" +
|
||||
"oded_region_name\030\001 \002(\014\022\022\n\ntable_name\030\002 \002" +
|
||||
"(\014\022\033\n\023log_sequence_number\030\003 \002(\004\022\022\n\nwrite" +
|
||||
"_time\030\004 \002(\004\022\035\n\ncluster_id\030\005 \001(\0132\005.UUIDB\002" +
|
||||
"\030\001\022\034\n\006scopes\030\006 \003(\0132\014.FamilyScope\022\032\n\022foll" +
|
||||
"owing_kv_count\030\007 \001(\r\022\032\n\013cluster_ids\030\010 \003(" +
|
||||
"\0132\005.UUID\"=\n\013FamilyScope\022\016\n\006family\030\001 \002(\014\022" +
|
||||
"\036\n\nscope_type\030\002 \002(\0162\n.ScopeType\"\251\001\n\024Comp" +
|
||||
"actionDescriptor\022\022\n\ntable_name\030\001 \002(\014\022\033\n\023",
|
||||
"encoded_region_name\030\002 \002(\014\022\023\n\013family_name" +
|
||||
"\030\003 \002(\014\022\030\n\020compaction_input\030\004 \003(\t\022\031\n\021comp" +
|
||||
"action_output\030\005 \003(\t\022\026\n\016store_home_dir\030\006 " +
|
||||
"\002(\t\"\014\n\nWALTrailer*F\n\tScopeType\022\033\n\027REPLIC" +
|
||||
"ATION_SCOPE_LOCAL\020\000\022\034\n\030REPLICATION_SCOPE" +
|
||||
"_GLOBAL\020\001B?\n*org.apache.hadoop.hbase.pro" +
|
||||
"tobuf.generatedB\tWALProtosH\001\210\001\000\240\001\001"
|
||||
"\0132\005.UUID\022\022\n\nnonceGroup\030\t \001(\004\022\r\n\005nonce\030\n " +
|
||||
"\001(\004\"=\n\013FamilyScope\022\016\n\006family\030\001 \002(\014\022\036\n\nsc" +
|
||||
"ope_type\030\002 \002(\0162\n.ScopeType\"\251\001\n\024Compactio",
|
||||
"nDescriptor\022\022\n\ntable_name\030\001 \002(\014\022\033\n\023encod" +
|
||||
"ed_region_name\030\002 \002(\014\022\023\n\013family_name\030\003 \002(" +
|
||||
"\014\022\030\n\020compaction_input\030\004 \003(\t\022\031\n\021compactio" +
|
||||
"n_output\030\005 \003(\t\022\026\n\016store_home_dir\030\006 \002(\t\"\014" +
|
||||
"\n\nWALTrailer*F\n\tScopeType\022\033\n\027REPLICATION" +
|
||||
"_SCOPE_LOCAL\020\000\022\034\n\030REPLICATION_SCOPE_GLOB" +
|
||||
"AL\020\001B?\n*org.apache.hadoop.hbase.protobuf" +
|
||||
".generatedB\tWALProtosH\001\210\001\000\240\001\001"
|
||||
};
|
||||
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
|
||||
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
|
||||
|
@ -4845,7 +5026,7 @@ public final class WALProtos {
|
|||
internal_static_WALKey_fieldAccessorTable = new
|
||||
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
|
||||
internal_static_WALKey_descriptor,
|
||||
new java.lang.String[] { "EncodedRegionName", "TableName", "LogSequenceNumber", "WriteTime", "ClusterId", "Scopes", "FollowingKvCount", "ClusterIds", });
|
||||
new java.lang.String[] { "EncodedRegionName", "TableName", "LogSequenceNumber", "WriteTime", "ClusterId", "Scopes", "FollowingKvCount", "ClusterIds", "NonceGroup", "Nonce", });
|
||||
internal_static_FamilyScope_descriptor =
|
||||
getDescriptor().getMessageTypes().get(2);
|
||||
internal_static_FamilyScope_fieldAccessorTable = new
|
||||
|
|
|
@ -133,6 +133,8 @@ message MutationProto {
|
|||
// 'cell' field above which is non-null when the cells are pb'd.
|
||||
optional int32 associated_cell_count = 8;
|
||||
|
||||
optional uint64 nonce = 9;
|
||||
|
||||
enum Durability {
|
||||
USE_DEFAULT = 0;
|
||||
SKIP_WAL = 1;
|
||||
|
@ -182,6 +184,7 @@ message MutateRequest {
|
|||
required RegionSpecifier region = 1;
|
||||
required MutationProto mutation = 2;
|
||||
optional Condition condition = 3;
|
||||
optional uint64 nonce_group = 4;
|
||||
}
|
||||
|
||||
message MutateResponse {
|
||||
|
@ -345,6 +348,7 @@ message RegionActionResult {
|
|||
*/
|
||||
message MultiRequest {
|
||||
repeated RegionAction regionAction = 1;
|
||||
optional uint64 nonceGroup = 2;
|
||||
}
|
||||
|
||||
message MultiResponse {
|
||||
|
|
|
@ -30,6 +30,8 @@ message MultiRowMutationProcessorResponse{
|
|||
|
||||
message MutateRowsRequest {
|
||||
repeated MutationProto mutation_request = 1;
|
||||
optional uint64 nonce_group = 2;
|
||||
optional uint64 nonce = 3;
|
||||
}
|
||||
|
||||
message MutateRowsResponse {
|
||||
|
|
|
@ -30,6 +30,8 @@ message ProcessRequest {
|
|||
required string row_processor_class_name = 1;
|
||||
optional string row_processor_initializer_message_name = 2;
|
||||
optional bytes row_processor_initializer_message = 3;
|
||||
optional uint64 nonce_group = 4;
|
||||
optional uint64 nonce = 5;
|
||||
}
|
||||
|
||||
message ProcessResponse {
|
||||
|
|
|
@ -43,14 +43,19 @@ message WALKey {
|
|||
|
||||
repeated FamilyScope scopes = 6;
|
||||
optional uint32 following_kv_count = 7;
|
||||
|
||||
/*
|
||||
This field contains the list of clusters that have
|
||||
consumed the change
|
||||
*/
|
||||
repeated UUID cluster_ids = 8;
|
||||
|
||||
optional uint64 nonceGroup = 9;
|
||||
optional uint64 nonce = 10;
|
||||
|
||||
/*
|
||||
optional CustomEntryType custom_entry_type = 9;
|
||||
|
||||
|
||||
enum CustomEntryType {
|
||||
COMPACTION = 0;
|
||||
}
|
||||
|
|
|
@ -380,4 +380,9 @@ public class CoprocessorHConnection implements HConnection {
|
|||
public boolean isDeadServer(ServerName serverName) {
|
||||
return delegate.isDeadServer(serverName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NonceGenerator getNonceGenerator() {
|
||||
return null; // don't use nonces for coprocessor connection
|
||||
}
|
||||
}
|
|
@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
import org.apache.hadoop.hbase.Coprocessor;
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.protobuf.ResponseConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessResponse;
|
||||
|
@ -68,7 +69,9 @@ extends RowProcessorService implements CoprocessorService, Coprocessor {
|
|||
try {
|
||||
RowProcessor<S,T> processor = constructRowProcessorFromRequest(request);
|
||||
HRegion region = env.getRegion();
|
||||
region.processRowsWithLocks(processor);
|
||||
long nonceGroup = request.hasNonceGroup() ? request.getNonceGroup() : HConstants.NO_NONCE;
|
||||
long nonce = request.hasNonce() ? request.getNonce() : HConstants.NO_NONCE;
|
||||
region.processRowsWithLocks(processor, nonceGroup, nonce);
|
||||
T result = processor.getResult();
|
||||
ProcessResponse.Builder b = ProcessResponse.newBuilder();
|
||||
b.setRowProcessorResult(result.toByteString());
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
import org.apache.hadoop.hbase.Coprocessor;
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.client.Mutation;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
|
@ -80,7 +81,7 @@ public class MultiRowMutationEndpoint extends MultiRowMutationService implements
|
|||
CoprocessorService, Coprocessor {
|
||||
private RegionCoprocessorEnvironment env;
|
||||
@Override
|
||||
public void mutateRows(RpcController controller, MutateRowsRequest request,
|
||||
public void mutateRows(RpcController controller, MutateRowsRequest request,
|
||||
RpcCallback<MutateRowsResponse> done) {
|
||||
MutateRowsResponse response = MutateRowsResponse.getDefaultInstance();
|
||||
try {
|
||||
|
@ -110,7 +111,9 @@ CoprocessorService, Coprocessor {
|
|||
rowsToLock.add(m.getRow());
|
||||
}
|
||||
// call utility method on region
|
||||
env.getRegion().mutateRowsWithLocks(mutations, rowsToLock);
|
||||
long nonceGroup = request.hasNonceGroup() ? request.getNonceGroup() : HConstants.NO_NONCE;
|
||||
long nonce = request.hasNonce() ? request.getNonce() : HConstants.NO_NONCE;
|
||||
env.getRegion().mutateRowsWithLocks(mutations, rowsToLock, nonceGroup, nonce);
|
||||
} catch (IOException e) {
|
||||
ResponseConverter.setControllerException(controller, e);
|
||||
}
|
||||
|
|
|
@ -31,6 +31,7 @@ import java.util.UUID;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.io.SizedCellScanner;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
|
@ -86,6 +87,7 @@ public class ReplicationProtbufUtil {
|
|||
HBaseProtos.UUID.Builder uuidBuilder = HBaseProtos.UUID.newBuilder();
|
||||
for (HLog.Entry entry: entries) {
|
||||
entryBuilder.clear();
|
||||
// TODO: this duplicates a lot in HLogKey#getBuilder
|
||||
WALProtos.WALKey.Builder keyBuilder = entryBuilder.getKeyBuilder();
|
||||
HLogKey key = entry.getKey();
|
||||
keyBuilder.setEncodedRegionName(
|
||||
|
@ -93,6 +95,12 @@ public class ReplicationProtbufUtil {
|
|||
keyBuilder.setTableName(ZeroCopyLiteralByteString.wrap(key.getTablename().getName()));
|
||||
keyBuilder.setLogSequenceNumber(key.getLogSeqNum());
|
||||
keyBuilder.setWriteTime(key.getWriteTime());
|
||||
if (key.getNonce() != HConstants.NO_NONCE) {
|
||||
keyBuilder.setNonce(key.getNonce());
|
||||
}
|
||||
if (key.getNonceGroup() != HConstants.NO_NONCE) {
|
||||
keyBuilder.setNonceGroup(key.getNonceGroup());
|
||||
}
|
||||
for(UUID clusterId : key.getClusterIds()) {
|
||||
uuidBuilder.setLeastSigBits(clusterId.getLeastSignificantBits());
|
||||
uuidBuilder.setMostSigBits(clusterId.getMostSignificantBits());
|
||||
|
|
|
@ -68,6 +68,7 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CompoundConfiguration;
|
||||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.DroppedSnapshotException;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
|
@ -122,6 +123,8 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
|
|||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter.MutationReplay;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
|
||||
|
@ -1897,7 +1900,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
* accumulating status codes and tracking the index at which processing
|
||||
* is proceeding.
|
||||
*/
|
||||
private static class BatchOperationInProgress<T> {
|
||||
private abstract static class BatchOperationInProgress<T> {
|
||||
T[] operations;
|
||||
int nextIndexToProcess = 0;
|
||||
OperationStatus[] retCodeDetails;
|
||||
|
@ -1910,11 +1913,84 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
Arrays.fill(this.retCodeDetails, OperationStatus.NOT_RUN);
|
||||
}
|
||||
|
||||
public abstract Mutation getMutation(int index);
|
||||
public abstract long getNonceGroup(int index);
|
||||
public abstract long getNonce(int index);
|
||||
/** This method is potentially expensive and should only be used for non-replay CP path. */
|
||||
public abstract Mutation[] getMutationsForCoprocs();
|
||||
public abstract boolean isInReplay();
|
||||
|
||||
public boolean isDone() {
|
||||
return nextIndexToProcess == operations.length;
|
||||
}
|
||||
}
|
||||
|
||||
private static class MutationBatch extends BatchOperationInProgress<Mutation> {
|
||||
private long nonceGroup;
|
||||
private long nonce;
|
||||
public MutationBatch(Mutation[] operations, long nonceGroup, long nonce) {
|
||||
super(operations);
|
||||
this.nonceGroup = nonceGroup;
|
||||
this.nonce = nonce;
|
||||
}
|
||||
|
||||
public Mutation getMutation(int index) {
|
||||
return this.operations[index];
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNonceGroup(int index) {
|
||||
return nonceGroup;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNonce(int index) {
|
||||
return nonce;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Mutation[] getMutationsForCoprocs() {
|
||||
return this.operations;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isInReplay() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
private static class ReplayBatch extends BatchOperationInProgress<HLogSplitter.MutationReplay> {
|
||||
public ReplayBatch(MutationReplay[] operations) {
|
||||
super(operations);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Mutation getMutation(int index) {
|
||||
return this.operations[index].mutation;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNonceGroup(int index) {
|
||||
return this.operations[index].nonceGroup;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNonce(int index) {
|
||||
return this.operations[index].nonce;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Mutation[] getMutationsForCoprocs() {
|
||||
assert false;
|
||||
throw new RuntimeException("Should not be called for replay batch");
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isInReplay() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform a batch of mutations.
|
||||
* It supports only Put and Delete mutations and will ignore other types passed.
|
||||
|
@ -1923,8 +1999,28 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
* OperationStatusCode and the exceptionMessage if any.
|
||||
* @throws IOException
|
||||
*/
|
||||
public OperationStatus[] batchMutate(
|
||||
Mutation[] mutations, long nonceGroup, long nonce) throws IOException {
|
||||
// As it stands, this is used for 3 things
|
||||
// * batchMutate with single mutation - put/delete, separate or from checkAndMutate.
|
||||
// * coprocessor calls (see ex. BulkDeleteEndpoint).
|
||||
// So nonces are not really ever used by HBase. They could be by coprocs, and checkAnd...
|
||||
return batchMutate(new MutationBatch(mutations, nonceGroup, nonce));
|
||||
}
|
||||
|
||||
public OperationStatus[] batchMutate(Mutation[] mutations) throws IOException {
|
||||
return batchMutate(mutations, false);
|
||||
return batchMutate(mutations, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Replay a batch of mutations.
|
||||
* @param mutations mutations to replay.
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
public OperationStatus[] batchReplay(HLogSplitter.MutationReplay[] mutations)
|
||||
throws IOException {
|
||||
return batchMutate(new ReplayBatch(mutations));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1935,21 +2031,16 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
* OperationStatusCode and the exceptionMessage if any.
|
||||
* @throws IOException
|
||||
*/
|
||||
OperationStatus[] batchMutate(Mutation[] mutations, boolean isReplay)
|
||||
throws IOException {
|
||||
BatchOperationInProgress<Mutation> batchOp =
|
||||
new BatchOperationInProgress<Mutation>(mutations);
|
||||
|
||||
OperationStatus[] batchMutate(BatchOperationInProgress<?> batchOp) throws IOException {
|
||||
boolean initialized = false;
|
||||
|
||||
while (!batchOp.isDone()) {
|
||||
if (!isReplay) {
|
||||
if (!batchOp.isInReplay()) {
|
||||
checkReadOnly();
|
||||
}
|
||||
checkResources();
|
||||
|
||||
long newSize;
|
||||
if (isReplay) {
|
||||
if (batchOp.isInReplay()) {
|
||||
startRegionOperation(Operation.REPLAY_BATCH_MUTATE);
|
||||
} else {
|
||||
startRegionOperation(Operation.BATCH_MUTATE);
|
||||
|
@ -1957,13 +2048,13 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
|
||||
try {
|
||||
if (!initialized) {
|
||||
if (!isReplay) {
|
||||
if (!batchOp.isInReplay()) {
|
||||
this.writeRequestsCount.increment();
|
||||
doPreMutationHook(batchOp);
|
||||
}
|
||||
initialized = true;
|
||||
}
|
||||
long addedSize = doMiniBatchMutation(batchOp, isReplay);
|
||||
long addedSize = doMiniBatchMutation(batchOp);
|
||||
newSize = this.addAndGetGlobalMemstoreSize(addedSize);
|
||||
} finally {
|
||||
closeRegionOperation();
|
||||
|
@ -1976,13 +2067,13 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
|
||||
|
||||
private void doPreMutationHook(BatchOperationInProgress<Mutation> batchOp)
|
||||
private void doPreMutationHook(BatchOperationInProgress<?> batchOp)
|
||||
throws IOException {
|
||||
/* Run coprocessor pre hook outside of locks to avoid deadlock */
|
||||
WALEdit walEdit = new WALEdit();
|
||||
if (coprocessorHost != null) {
|
||||
for (int i = 0 ; i < batchOp.operations.length; i++) {
|
||||
Mutation m = batchOp.operations[i];
|
||||
Mutation m = batchOp.getMutation(i);
|
||||
if (m instanceof Put) {
|
||||
if (coprocessorHost.prePut((Put) m, walEdit, m.getDurability())) {
|
||||
// pre hook says skip this Put
|
||||
|
@ -2011,9 +2102,8 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private long doMiniBatchMutation(BatchOperationInProgress<Mutation> batchOp,
|
||||
boolean isInReplay) throws IOException {
|
||||
|
||||
private long doMiniBatchMutation(BatchOperationInProgress<?> batchOp) throws IOException {
|
||||
boolean isInReplay = batchOp.isInReplay();
|
||||
// variable to note if all Put items are for the same CF -- metrics related
|
||||
boolean putsCfSetConsistent = true;
|
||||
//The set of columnFamilies first seen for Put.
|
||||
|
@ -2023,6 +2113,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
//The set of columnFamilies first seen for Delete.
|
||||
Set<byte[]> deletesCfSet = null;
|
||||
|
||||
long currentNonceGroup = HConstants.NO_NONCE, currentNonce = HConstants.NO_NONCE;
|
||||
WALEdit walEdit = new WALEdit(isInReplay);
|
||||
MultiVersionConsistencyControl.WriteEntry w = null;
|
||||
long txid = 0;
|
||||
|
@ -2046,7 +2137,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
int numReadyToWrite = 0;
|
||||
long now = EnvironmentEdgeManager.currentTimeMillis();
|
||||
while (lastIndexExclusive < batchOp.operations.length) {
|
||||
Mutation mutation = batchOp.operations[lastIndexExclusive];
|
||||
Mutation mutation = batchOp.getMutation(lastIndexExclusive);
|
||||
boolean isPutMutation = mutation instanceof Put;
|
||||
|
||||
Map<byte[], List<Cell>> familyMap = mutation.getFamilyCellMap();
|
||||
|
@ -2145,7 +2236,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
if (batchOp.retCodeDetails[i].getOperationStatusCode()
|
||||
!= OperationStatusCode.NOT_RUN) continue;
|
||||
|
||||
Mutation mutation = batchOp.operations[i];
|
||||
Mutation mutation = batchOp.getMutation(i);
|
||||
if (mutation instanceof Put) {
|
||||
updateKVTimestamps(familyMaps[i].values(), byteNow);
|
||||
noOfPuts++;
|
||||
|
@ -2167,7 +2258,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
// calling the pre CP hook for batch mutation
|
||||
if (!isInReplay && coprocessorHost != null) {
|
||||
MiniBatchOperationInProgress<Mutation> miniBatchOp =
|
||||
new MiniBatchOperationInProgress<Mutation>(batchOp.operations,
|
||||
new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
|
||||
batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
|
||||
if (coprocessorHost.preBatchMutate(miniBatchOp)) return 0L;
|
||||
}
|
||||
|
@ -2193,6 +2284,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
// ------------------------------------
|
||||
// STEP 4. Build WAL edit
|
||||
// ----------------------------------
|
||||
boolean hasWalAppends = false;
|
||||
Durability durability = Durability.USE_DEFAULT;
|
||||
for (int i = firstIndex; i < lastIndexExclusive; i++) {
|
||||
// Skip puts that were determined to be invalid during preprocessing
|
||||
|
@ -2202,7 +2294,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
|
||||
|
||||
Mutation m = batchOp.operations[i];
|
||||
Mutation m = batchOp.getMutation(i);
|
||||
Durability tmpDur = getEffectiveDurability(m.getDurability());
|
||||
if (tmpDur.ordinal() > durability.ordinal()) {
|
||||
durability = tmpDur;
|
||||
|
@ -2212,6 +2304,22 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
continue;
|
||||
}
|
||||
|
||||
long nonceGroup = batchOp.getNonceGroup(i), nonce = batchOp.getNonce(i);
|
||||
// In replay, the batch may contain multiple nonces. If so, write WALEdit for each.
|
||||
// Given how nonces are originally written, these should be contiguous.
|
||||
// txid should always increase, so having the last one is ok.
|
||||
if (nonceGroup != currentNonceGroup || nonce != currentNonce) {
|
||||
if (walEdit.size() > 0) {
|
||||
assert isInReplay;
|
||||
txid = this.log.appendNoSync(this.getRegionInfo(), htableDescriptor.getTableName(),
|
||||
walEdit, m.getClusterIds(), now, htableDescriptor, this.sequenceId, true,
|
||||
currentNonceGroup, currentNonce);
|
||||
hasWalAppends = true;
|
||||
}
|
||||
currentNonceGroup = nonceGroup;
|
||||
currentNonce = nonce;
|
||||
}
|
||||
|
||||
// Add WAL edits by CP
|
||||
WALEdit fromCP = batchOp.walEditsFromCoprocessors[i];
|
||||
if (fromCP != null) {
|
||||
|
@ -2223,12 +2331,14 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
|
||||
// -------------------------
|
||||
// STEP 5. Append the edit to WAL. Do not sync wal.
|
||||
// STEP 5. Append the final edit to WAL. Do not sync wal.
|
||||
// -------------------------
|
||||
Mutation mutation = batchOp.operations[firstIndex];
|
||||
Mutation mutation = batchOp.getMutation(firstIndex);
|
||||
if (walEdit.size() > 0) {
|
||||
txid = this.log.appendNoSync(this.getRegionInfo(), this.htableDescriptor.getTableName(),
|
||||
walEdit, mutation.getClusterIds(), now, this.htableDescriptor, this.sequenceId);
|
||||
walEdit, mutation.getClusterIds(), now, this.htableDescriptor, this.sequenceId,
|
||||
true, currentNonceGroup, currentNonce);
|
||||
hasWalAppends = true;
|
||||
}
|
||||
|
||||
// -------------------------------
|
||||
|
@ -2243,14 +2353,14 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
// -------------------------
|
||||
// STEP 7. Sync wal.
|
||||
// -------------------------
|
||||
if (walEdit.size() > 0) {
|
||||
if (hasWalAppends) {
|
||||
syncOrDefer(txid, durability);
|
||||
}
|
||||
walSyncSuccessful = true;
|
||||
// calling the post CP hook for batch mutation
|
||||
if (!isInReplay && coprocessorHost != null) {
|
||||
MiniBatchOperationInProgress<Mutation> miniBatchOp =
|
||||
new MiniBatchOperationInProgress<Mutation>(batchOp.operations,
|
||||
new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
|
||||
batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
|
||||
coprocessorHost.postBatchMutate(miniBatchOp);
|
||||
}
|
||||
|
@ -2274,7 +2384,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
!= OperationStatusCode.SUCCESS) {
|
||||
continue;
|
||||
}
|
||||
Mutation m = batchOp.operations[i];
|
||||
Mutation m = batchOp.getMutation(i);
|
||||
if (m instanceof Put) {
|
||||
coprocessorHost.postPut((Put) m, walEdit, m.getDurability());
|
||||
} else {
|
||||
|
@ -2362,10 +2472,10 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
boolean isPut = w instanceof Put;
|
||||
if (!isPut && !(w instanceof Delete))
|
||||
throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action must " +
|
||||
"be Put or Delete");
|
||||
"be Put or Delete");
|
||||
if (!Bytes.equals(row, w.getRow())) {
|
||||
throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action's " +
|
||||
"getRow must match the passed row");
|
||||
"getRow must match the passed row");
|
||||
}
|
||||
|
||||
startRegionOperation();
|
||||
|
@ -2435,9 +2545,10 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
}
|
||||
|
||||
private void doBatchMutate(Mutation mutation) throws IOException,
|
||||
org.apache.hadoop.hbase.DoNotRetryIOException {
|
||||
OperationStatus[] batchMutate = this.batchMutate(new Mutation[] { mutation });
|
||||
private void doBatchMutate(Mutation mutation) throws IOException, DoNotRetryIOException {
|
||||
// Currently this is only called for puts and deletes, so no nonces.
|
||||
OperationStatus[] batchMutate = this.batchMutate(new Mutation[] { mutation },
|
||||
HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||
if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) {
|
||||
throw new FailedSanityCheckException(batchMutate[0].getExceptionMsg());
|
||||
} else if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.BAD_FAMILY)) {
|
||||
|
@ -2628,7 +2739,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
* called when a Put/Delete has updated memstore but subequently fails to update
|
||||
* the wal. This method is then invoked to rollback the memstore.
|
||||
*/
|
||||
private void rollbackMemstore(BatchOperationInProgress<Mutation> batchOp,
|
||||
private void rollbackMemstore(BatchOperationInProgress<?> batchOp,
|
||||
Map<byte[], List<Cell>>[] familyMaps,
|
||||
int start, int end) {
|
||||
int kvsRolledback = 0;
|
||||
|
@ -2901,6 +3012,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
HLog.Entry entry;
|
||||
Store store = null;
|
||||
boolean reported_once = false;
|
||||
ServerNonceManager ng = this.rsServices == null ? null : this.rsServices.getNonceManager();
|
||||
|
||||
try {
|
||||
// How many edits seen before we check elapsed time
|
||||
|
@ -2916,6 +3028,10 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
HLogKey key = entry.getKey();
|
||||
WALEdit val = entry.getEdit();
|
||||
|
||||
if (ng != null) { // some test, or nonces disabled
|
||||
ng.reportOperationFromWal(key.getNonceGroup(), key.getNonce(), key.getWriteTime());
|
||||
}
|
||||
|
||||
if (reporter != null) {
|
||||
intervalEdits += val.size();
|
||||
if (intervalEdits >= interval) {
|
||||
|
@ -4395,35 +4511,47 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
|
||||
public void mutateRow(RowMutations rm) throws IOException {
|
||||
// Don't need nonces here - RowMutations only supports puts and deletes
|
||||
mutateRowsWithLocks(rm.getMutations(), Collections.singleton(rm.getRow()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform atomic mutations within the region w/o nonces.
|
||||
* See {@link #mutateRowsWithLocks(Collection, Collection, long, long)}
|
||||
*/
|
||||
public void mutateRowsWithLocks(Collection<Mutation> mutations,
|
||||
Collection<byte[]> rowsToLock) throws IOException {
|
||||
mutateRowsWithLocks(mutations, rowsToLock, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform atomic mutations within the region.
|
||||
* @param mutations The list of mutations to perform.
|
||||
* <code>mutations</code> can contain operations for multiple rows.
|
||||
* Caller has to ensure that all rows are contained in this region.
|
||||
* @param rowsToLock Rows to lock
|
||||
* @param nonceGroup Optional nonce group of the operation (client Id)
|
||||
* @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence")
|
||||
* If multiple rows are locked care should be taken that
|
||||
* <code>rowsToLock</code> is sorted in order to avoid deadlocks.
|
||||
* @throws IOException
|
||||
*/
|
||||
public void mutateRowsWithLocks(Collection<Mutation> mutations,
|
||||
Collection<byte[]> rowsToLock) throws IOException {
|
||||
|
||||
MultiRowMutationProcessor proc =
|
||||
new MultiRowMutationProcessor(mutations, rowsToLock);
|
||||
processRowsWithLocks(proc, -1);
|
||||
Collection<byte[]> rowsToLock, long nonceGroup, long nonce) throws IOException {
|
||||
MultiRowMutationProcessor proc = new MultiRowMutationProcessor(mutations, rowsToLock);
|
||||
processRowsWithLocks(proc, -1, nonceGroup, nonce);
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs atomic multiple reads and writes on a given row.
|
||||
*
|
||||
* @param processor The object defines the reads and writes to a row.
|
||||
* @param nonceGroup Optional nonce group of the operation (client Id)
|
||||
* @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence")
|
||||
*/
|
||||
public void processRowsWithLocks(RowProcessor<?,?> processor)
|
||||
public void processRowsWithLocks(RowProcessor<?,?> processor, long nonceGroup, long nonce)
|
||||
throws IOException {
|
||||
processRowsWithLocks(processor, rowProcessorTimeout);
|
||||
processRowsWithLocks(processor, rowProcessorTimeout, nonceGroup, nonce);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -4432,9 +4560,11 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
* @param processor The object defines the reads and writes to a row.
|
||||
* @param timeout The timeout of the processor.process() execution
|
||||
* Use a negative number to switch off the time bound
|
||||
* @param nonceGroup Optional nonce group of the operation (client Id)
|
||||
* @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence")
|
||||
*/
|
||||
public void processRowsWithLocks(RowProcessor<?,?> processor, long timeout)
|
||||
throws IOException {
|
||||
public void processRowsWithLocks(RowProcessor<?,?> processor, long timeout,
|
||||
long nonceGroup, long nonce) throws IOException {
|
||||
|
||||
for (byte[] row : processor.getRowsToLock()) {
|
||||
checkRow(row, "processRowsWithLocks");
|
||||
|
@ -4506,7 +4636,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
if (!walEdit.isEmpty()) {
|
||||
txid = this.log.appendNoSync(this.getRegionInfo(),
|
||||
this.htableDescriptor.getTableName(), walEdit, processor.getClusterIds(), now,
|
||||
this.htableDescriptor, this.sequenceId);
|
||||
this.htableDescriptor, this.sequenceId, true, nonceGroup, nonce);
|
||||
}
|
||||
// 8. Release region lock
|
||||
if (locked) {
|
||||
|
@ -4609,6 +4739,10 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
}
|
||||
|
||||
public Result append(Append append) throws IOException {
|
||||
return append(append, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||
}
|
||||
|
||||
// TODO: There's a lot of boiler plate code identical
|
||||
// to increment... See how to better unify that.
|
||||
/**
|
||||
|
@ -4618,7 +4752,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
* @return new keyvalues after increment
|
||||
* @throws IOException
|
||||
*/
|
||||
public Result append(Append append)
|
||||
public Result append(Append append, long nonceGroup, long nonce)
|
||||
throws IOException {
|
||||
byte[] row = append.getRow();
|
||||
checkRow(row, "append");
|
||||
|
@ -4749,7 +4883,8 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
// as a Put.
|
||||
txid = this.log.appendNoSync(this.getRegionInfo(),
|
||||
this.htableDescriptor.getTableName(), walEdits, new ArrayList<UUID>(),
|
||||
EnvironmentEdgeManager.currentTimeMillis(), this.htableDescriptor, this.sequenceId);
|
||||
EnvironmentEdgeManager.currentTimeMillis(), this.htableDescriptor, this.sequenceId,
|
||||
true, nonceGroup, nonce);
|
||||
} else {
|
||||
recordMutationWithoutWal(append.getFamilyCellMap());
|
||||
}
|
||||
|
@ -4801,13 +4936,17 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
return append.isReturnResults() ? Result.create(allKVs) : null;
|
||||
}
|
||||
|
||||
public Result increment(Increment increment) throws IOException {
|
||||
return increment(increment, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform one or more increment operations on a row.
|
||||
* @param increment
|
||||
* @return new keyvalues after increment
|
||||
* @throws IOException
|
||||
*/
|
||||
public Result increment(Increment increment)
|
||||
public Result increment(Increment increment, long nonceGroup, long nonce)
|
||||
throws IOException {
|
||||
byte [] row = increment.getRow();
|
||||
checkRow(row, "increment");
|
||||
|
@ -4923,7 +5062,8 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
// as a Put.
|
||||
txid = this.log.appendNoSync(this.getRegionInfo(),
|
||||
this.htableDescriptor.getTableName(), walEdits, new ArrayList<UUID>(),
|
||||
EnvironmentEdgeManager.currentTimeMillis(), this.htableDescriptor, this.sequenceId);
|
||||
EnvironmentEdgeManager.currentTimeMillis(), this.htableDescriptor, this.sequenceId,
|
||||
true, nonceGroup, nonce);
|
||||
} else {
|
||||
recordMutationWithoutWal(increment.getFamilyCellMap());
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.lang.Thread.UncaughtExceptionHandler;
|
||||
import java.lang.annotation.Retention;
|
||||
import java.lang.annotation.RetentionPolicy;
|
||||
|
@ -93,6 +94,7 @@ import org.apache.hadoop.hbase.client.RowMutations;
|
|||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
|
||||
import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
|
||||
import org.apache.hadoop.hbase.exceptions.OperationConflictException;
|
||||
import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
|
||||
import org.apache.hadoop.hbase.exceptions.RegionMovedException;
|
||||
import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
|
||||
|
@ -441,6 +443,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
/** The health check chore. */
|
||||
private HealthCheckChore healthCheckChore;
|
||||
|
||||
/** The nonce manager chore. */
|
||||
private Chore nonceManagerChore;
|
||||
|
||||
/**
|
||||
* The server name the Master sees us as. Its made from the hostname the
|
||||
* master passes us, port, and server startcode. Gets set after registration
|
||||
|
@ -490,6 +495,26 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
// Table level lock manager for locking for region operations
|
||||
private TableLockManager tableLockManager;
|
||||
|
||||
/**
|
||||
* Nonce manager. Nonces are used to make operations like increment and append idempotent
|
||||
* in the case where client doesn't receive the response from a successful operation and
|
||||
* retries. We track the successful ops for some time via a nonce sent by client and handle
|
||||
* duplicate operations (currently, by failing them; in future we might use MVCC to return
|
||||
* result). Nonces are also recovered from WAL during, recovery; however, the caveats (from
|
||||
* HBASE-3787) are:
|
||||
* - WAL recovery is optimized, and under high load we won't read nearly nonce-timeout worth
|
||||
* of past records. If we don't read the records, we don't read and recover the nonces.
|
||||
* Some WALs within nonce-timeout at recovery may not even be present due to rolling/cleanup.
|
||||
* - There's no WAL recovery during normal region move, so nonces will not be transfered.
|
||||
* We can have separate additional "Nonce WAL". It will just contain bunch of numbers and
|
||||
* won't be flushed on main path - because WAL itself also contains nonces, if we only flush
|
||||
* it before memstore flush, for a given nonce we will either see it in the WAL (if it was
|
||||
* never flushed to disk, it will be part of recovery), or we'll see it as part of the nonce
|
||||
* log (or both occasionally, which doesn't matter). Nonce log file can be deleted after the
|
||||
* latest nonce in it expired. It can also be recovered during move.
|
||||
*/
|
||||
private final ServerNonceManager nonceManager;
|
||||
|
||||
private UserProvider userProvider;
|
||||
|
||||
/**
|
||||
|
@ -529,6 +554,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
|
||||
this.sleeper = new Sleeper(this.msgInterval, this);
|
||||
|
||||
boolean isNoncesEnabled = conf.getBoolean(HConstants.HBASE_RS_NONCES_ENABLED, true);
|
||||
this.nonceManager = isNoncesEnabled ? new ServerNonceManager(this.conf) : null;
|
||||
|
||||
this.maxScannerResultSize = conf.getLong(
|
||||
HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
|
||||
HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
|
||||
|
@ -789,6 +817,11 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
// Create the thread to clean the moved regions list
|
||||
movedRegionsCleaner = MovedRegionsCleaner.createAndStart(this);
|
||||
|
||||
if (this.nonceManager != null) {
|
||||
// Create the chore that cleans up nonces.
|
||||
nonceManagerChore = this.nonceManager.createCleanupChore(this);
|
||||
}
|
||||
|
||||
// Setup RPC client for master communication
|
||||
rpcClient = new RpcClient(conf, clusterId, new InetSocketAddress(
|
||||
this.isa.getAddress(), 0));
|
||||
|
@ -910,6 +943,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
if (this.healthCheckChore != null) {
|
||||
this.healthCheckChore.interrupt();
|
||||
}
|
||||
if (this.nonceManagerChore != null) {
|
||||
this.nonceManagerChore.interrupt();
|
||||
}
|
||||
|
||||
// Stop the snapshot handler, forcefully killing all running tasks
|
||||
try {
|
||||
|
@ -1556,8 +1592,11 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
Threads.setDaemonThreadRunning(this.periodicFlusher.getThread(), n +
|
||||
".periodicFlusher", uncaughtExceptionHandler);
|
||||
if (this.healthCheckChore != null) {
|
||||
Threads
|
||||
.setDaemonThreadRunning(this.healthCheckChore.getThread(), n + ".healthChecker",
|
||||
Threads.setDaemonThreadRunning(this.healthCheckChore.getThread(), n + ".healthChecker",
|
||||
uncaughtExceptionHandler);
|
||||
}
|
||||
if (this.nonceManagerChore != null) {
|
||||
Threads.setDaemonThreadRunning(this.nonceManagerChore.getThread(), n + ".nonceCleaner",
|
||||
uncaughtExceptionHandler);
|
||||
}
|
||||
|
||||
|
@ -1811,6 +1850,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
* have already been called.
|
||||
*/
|
||||
protected void join() {
|
||||
if (this.nonceManagerChore != null) {
|
||||
Threads.shutdown(this.nonceManagerChore.getThread());
|
||||
}
|
||||
Threads.shutdown(this.compactionChecker.getThread());
|
||||
Threads.shutdown(this.periodicFlusher.getThread());
|
||||
this.cacheFlusher.join();
|
||||
|
@ -2826,15 +2868,19 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
if (!region.getRegionInfo().isMetaTable()) {
|
||||
cacheFlusher.reclaimMemStoreMemory();
|
||||
}
|
||||
long nonceGroup = request.hasNonceGroup()
|
||||
? request.getNonceGroup() : HConstants.NO_NONCE;
|
||||
Result r = null;
|
||||
Boolean processed = null;
|
||||
MutationType type = mutation.getMutateType();
|
||||
switch (type) {
|
||||
case APPEND:
|
||||
r = append(region, mutation, cellScanner);
|
||||
// TODO: this doesn't actually check anything.
|
||||
r = append(region, mutation, cellScanner, nonceGroup);
|
||||
break;
|
||||
case INCREMENT:
|
||||
r = increment(region, mutation, cellScanner);
|
||||
// TODO: this doesn't actually check anything.
|
||||
r = increment(region, mutation, cellScanner, nonceGroup);
|
||||
break;
|
||||
case PUT:
|
||||
Put put = ProtobufUtil.toPut(mutation, cellScanner);
|
||||
|
@ -3251,6 +3297,8 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
CellScanner cellScanner = controller != null ? controller.cellScanner(): null;
|
||||
if (controller != null) controller.setCellScanner(null);
|
||||
|
||||
long nonceGroup = request.hasNonceGroup() ? request.getNonceGroup() : HConstants.NO_NONCE;
|
||||
|
||||
// this will contain all the cells that we need to return. It's created later, if needed.
|
||||
List<CellScannable> cellsToReturn = null;
|
||||
MultiResponse.Builder responseBuilder = MultiResponse.newBuilder();
|
||||
|
@ -3279,7 +3327,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
} else {
|
||||
// doNonAtomicRegionMutation manages the exception internally
|
||||
cellsToReturn = doNonAtomicRegionMutation(region, regionAction, cellScanner,
|
||||
regionActionResultBuilder, cellsToReturn);
|
||||
regionActionResultBuilder, cellsToReturn, nonceGroup);
|
||||
}
|
||||
responseBuilder.addRegionActionResult(regionActionResultBuilder.build());
|
||||
}
|
||||
|
@ -3303,7 +3351,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
*/
|
||||
private List<CellScannable> doNonAtomicRegionMutation(final HRegion region,
|
||||
final RegionAction actions, final CellScanner cellScanner,
|
||||
final RegionActionResult.Builder builder, List<CellScannable> cellsToReturn) {
|
||||
final RegionActionResult.Builder builder, List<CellScannable> cellsToReturn, long nonceGroup) {
|
||||
// Gather up CONTIGUOUS Puts and Deletes in this mutations List. Idea is that rather than do
|
||||
// one at a time, we instead pass them in batch. Be aware that the corresponding
|
||||
// ResultOrException instance that matches each Put or Delete is then added down in the
|
||||
|
@ -3326,10 +3374,10 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
}
|
||||
switch (type) {
|
||||
case APPEND:
|
||||
r = append(region, action.getMutation(), cellScanner);
|
||||
r = append(region, action.getMutation(), cellScanner, nonceGroup);
|
||||
break;
|
||||
case INCREMENT:
|
||||
r = increment(region, action.getMutation(), cellScanner);
|
||||
r = increment(region, action.getMutation(), cellScanner, nonceGroup);
|
||||
break;
|
||||
case PUT:
|
||||
case DELETE:
|
||||
|
@ -3862,12 +3910,18 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
entries.get(0).getKey().getEncodedRegionName().toStringUtf8());
|
||||
RegionCoprocessorHost coprocessorHost = region.getCoprocessorHost();
|
||||
List<Pair<HLogKey, WALEdit>> walEntries = new ArrayList<Pair<HLogKey, WALEdit>>();
|
||||
List<Pair<MutationType, Mutation>> mutations = new ArrayList<Pair<MutationType, Mutation>>();
|
||||
List<HLogSplitter.MutationReplay> mutations = new ArrayList<HLogSplitter.MutationReplay>();
|
||||
for (WALEntry entry : entries) {
|
||||
if (nonceManager != null) {
|
||||
long nonceGroup = entry.getKey().hasNonceGroup()
|
||||
? entry.getKey().getNonceGroup() : HConstants.NO_NONCE;
|
||||
long nonce = entry.getKey().hasNonce() ? entry.getKey().getNonce() : HConstants.NO_NONCE;
|
||||
nonceManager.reportOperationFromWal(nonceGroup, nonce, entry.getKey().getWriteTime());
|
||||
}
|
||||
Pair<HLogKey, WALEdit> walEntry = (coprocessorHost == null) ? null :
|
||||
new Pair<HLogKey, WALEdit>();
|
||||
List<Pair<MutationType, Mutation>> edits = HLogSplitter.getMutationsFromWALEntry(entry,
|
||||
cells, walEntry);
|
||||
List<HLogSplitter.MutationReplay> edits =
|
||||
HLogSplitter.getMutationsFromWALEntry(entry, cells, walEntry);
|
||||
if (coprocessorHost != null) {
|
||||
// Start coprocessor replay here. The coprocessor is for each WALEdit instead of a
|
||||
// KeyValue.
|
||||
|
@ -3882,7 +3936,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
}
|
||||
|
||||
if (!mutations.isEmpty()) {
|
||||
OperationStatus[] result = doBatchOp(region, mutations, true);
|
||||
OperationStatus[] result = doReplayBatchOp(region, mutations);
|
||||
// check if it's a partial success
|
||||
for (int i = 0; result != null && i < result.length; i++) {
|
||||
if (result[i] != OperationStatus.SUCCESS) {
|
||||
|
@ -3987,7 +4041,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
* @throws IOException
|
||||
*/
|
||||
protected Result append(final HRegion region,
|
||||
final MutationProto m, final CellScanner cellScanner) throws IOException {
|
||||
final MutationProto m, final CellScanner cellScanner, long nonceGroup) throws IOException {
|
||||
long before = EnvironmentEdgeManager.currentTimeMillis();
|
||||
Append append = ProtobufUtil.toAppend(m, cellScanner);
|
||||
Result r = null;
|
||||
|
@ -3995,7 +4049,14 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
r = region.getCoprocessorHost().preAppend(append);
|
||||
}
|
||||
if (r == null) {
|
||||
r = region.append(append);
|
||||
long nonce = startNonceOperation(m, nonceGroup);
|
||||
boolean success = false;
|
||||
try {
|
||||
r = region.append(append, nonceGroup, nonce);
|
||||
success = true;
|
||||
} finally {
|
||||
endNonceOperation(m, nonceGroup, success);
|
||||
}
|
||||
if (region.getCoprocessorHost() != null) {
|
||||
region.getCoprocessorHost().postAppend(append, r);
|
||||
}
|
||||
|
@ -4013,8 +4074,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
* @throws IOException
|
||||
*/
|
||||
protected Result increment(final HRegion region, final MutationProto mutation,
|
||||
final CellScanner cells)
|
||||
throws IOException {
|
||||
final CellScanner cells, long nonceGroup) throws IOException {
|
||||
long before = EnvironmentEdgeManager.currentTimeMillis();
|
||||
Increment increment = ProtobufUtil.toIncrement(mutation, cells);
|
||||
Result r = null;
|
||||
|
@ -4022,7 +4082,14 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
r = region.getCoprocessorHost().preIncrement(increment);
|
||||
}
|
||||
if (r == null) {
|
||||
r = region.increment(increment);
|
||||
long nonce = startNonceOperation(mutation, nonceGroup);
|
||||
boolean success = false;
|
||||
try {
|
||||
r = region.increment(increment, nonceGroup, nonce);
|
||||
success = true;
|
||||
} finally {
|
||||
endNonceOperation(mutation, nonceGroup, success);
|
||||
}
|
||||
if (region.getCoprocessorHost() != null) {
|
||||
r = region.getCoprocessorHost().postIncrement(increment, r);
|
||||
}
|
||||
|
@ -4031,6 +4098,49 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
return r;
|
||||
}
|
||||
|
||||
/**
|
||||
* Starts the nonce operation for a mutation, if needed.
|
||||
* @param mutation Mutation.
|
||||
* @param nonceGroup Nonce group from the request.
|
||||
* @returns Nonce used (can be NO_NONCE).
|
||||
*/
|
||||
private long startNonceOperation(final MutationProto mutation, long nonceGroup)
|
||||
throws IOException, OperationConflictException {
|
||||
if (nonceManager == null || !mutation.hasNonce()) return HConstants.NO_NONCE;
|
||||
boolean canProceed = false;
|
||||
try {
|
||||
canProceed = nonceManager.startOperation(nonceGroup, mutation.getNonce(), this);
|
||||
} catch (InterruptedException ex) {
|
||||
// Probably should not happen.
|
||||
throw new InterruptedIOException("Nonce start operation interrupted");
|
||||
}
|
||||
if (!canProceed) {
|
||||
// TODO: instead, we could convert append/increment to get w/mvcc
|
||||
String message = "The operation with nonce {" + nonceGroup + ", " + mutation.getNonce()
|
||||
+ "} on row [" + Bytes.toString(mutation.getRow().toByteArray())
|
||||
+ "] may have already completed";
|
||||
throw new OperationConflictException(message);
|
||||
}
|
||||
return mutation.getNonce();
|
||||
}
|
||||
|
||||
/**
|
||||
* Ends nonce operation for a mutation, if needed.
|
||||
* @param mutation Mutation.
|
||||
* @param nonceGroup Nonce group from the request. Always 0 in initial implementation.
|
||||
* @param success Whether the operation for this nonce has succeeded.
|
||||
*/
|
||||
private void endNonceOperation(final MutationProto mutation, long nonceGroup,
|
||||
boolean success) {
|
||||
if (nonceManager == null || !mutation.hasNonce()) return;
|
||||
nonceManager.endOperation(nonceGroup, mutation.getNonce(), success);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServerNonceManager getNonceManager() {
|
||||
return this.nonceManager;
|
||||
}
|
||||
|
||||
/**
|
||||
* Execute a list of Put/Delete mutations.
|
||||
*
|
||||
|
@ -4063,7 +4173,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
cacheFlusher.reclaimMemStoreMemory();
|
||||
}
|
||||
|
||||
OperationStatus codes[] = region.batchMutate(mArray, false);
|
||||
OperationStatus codes[] = region.batchMutate(mArray);
|
||||
for (i = 0; i < codes.length; i++) {
|
||||
int index = mutations.get(i).getIndex();
|
||||
Exception e = null;
|
||||
|
@ -4119,32 +4229,31 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
|
|||
* constructing MultiResponse to save a possible loop if caller doesn't need MultiResponse.
|
||||
* @param region
|
||||
* @param mutations
|
||||
* @param isReplay
|
||||
* @return an array of OperationStatus which internally contains the OperationStatusCode and the
|
||||
* exceptionMessage if any
|
||||
* @throws IOException
|
||||
*/
|
||||
protected OperationStatus [] doBatchOp(final HRegion region,
|
||||
final List<Pair<MutationType, Mutation>> mutations, boolean isReplay)
|
||||
throws IOException {
|
||||
Mutation[] mArray = new Mutation[mutations.size()];
|
||||
protected OperationStatus [] doReplayBatchOp(final HRegion region,
|
||||
final List<HLogSplitter.MutationReplay> mutations) throws IOException {
|
||||
HLogSplitter.MutationReplay[] mArray = new HLogSplitter.MutationReplay[mutations.size()];
|
||||
|
||||
long before = EnvironmentEdgeManager.currentTimeMillis();
|
||||
boolean batchContainsPuts = false, batchContainsDelete = false;
|
||||
try {
|
||||
int i = 0;
|
||||
for (Pair<MutationType, Mutation> m : mutations) {
|
||||
if (m.getFirst() == MutationType.PUT) {
|
||||
for (HLogSplitter.MutationReplay m : mutations) {
|
||||
if (m.type == MutationType.PUT) {
|
||||
batchContainsPuts = true;
|
||||
} else {
|
||||
batchContainsDelete = true;
|
||||
}
|
||||
mArray[i++] = m.getSecond();
|
||||
mArray[i++] = m;
|
||||
}
|
||||
requestCount.add(mutations.size());
|
||||
if (!region.getRegionInfo().isMetaTable()) {
|
||||
cacheFlusher.reclaimMemStoreMemory();
|
||||
}
|
||||
return region.batchMutate(mArray, isReplay);
|
||||
return region.batchReplay(mArray);
|
||||
} finally {
|
||||
long after = EnvironmentEdgeManager.currentTimeMillis();
|
||||
if (batchContainsPuts) {
|
||||
|
|
|
@ -115,4 +115,10 @@ public interface RegionServerServices
|
|||
* @return set of recovering regions on the hosting region server
|
||||
*/
|
||||
Map<String, HRegion> getRecoveringRegions();
|
||||
|
||||
/**
|
||||
* Only required for "old" log replay; if it's removed, remove this.
|
||||
* @return The RegionServer's NonceManager
|
||||
*/
|
||||
public ServerNonceManager getNonceManager();
|
||||
}
|
||||
|
|
|
@ -0,0 +1,280 @@
|
|||
/**
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.util.Date;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Chore;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Implementation of nonce manager that stores nonces in a hash map and cleans them up after
|
||||
* some time; if nonce group/client ID is supplied, nonces are stored by client ID.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ServerNonceManager {
|
||||
public static final String HASH_NONCE_GRACE_PERIOD_KEY = "hbase.server.hashNonce.gracePeriod";
|
||||
private static final Log LOG = LogFactory.getLog(ServerNonceManager.class);
|
||||
|
||||
/** The time to wait in an extremely unlikely case of a conflict with a running op.
|
||||
* Only here so that tests could override it and not wait. */
|
||||
private int conflictWaitIterationMs = 30000;
|
||||
|
||||
private static final SimpleDateFormat tsFormat = new SimpleDateFormat("HH:mm:ss.SSS");
|
||||
|
||||
// This object is used to synchronize on in case of collisions, and for cleanup.
|
||||
private static class OperationContext {
|
||||
static final int DONT_PROCEED = 0;
|
||||
static final int PROCEED = 1;
|
||||
static final int WAIT = 2;
|
||||
|
||||
// 0..1 - state, 2..2 - whether anyone is waiting, 3.. - ts of last activity
|
||||
private long data = 0;
|
||||
private static final long STATE_BITS = 3;
|
||||
private static final long WAITING_BIT = 4;
|
||||
private static final long ALL_FLAG_BITS = WAITING_BIT | STATE_BITS;
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "[state " + getState() + ", hasWait " + hasWait() + ", activity "
|
||||
+ tsFormat.format(new Date(getActivityTime())) + "]";
|
||||
}
|
||||
|
||||
public OperationContext() {
|
||||
setState(WAIT);
|
||||
reportActivity();
|
||||
}
|
||||
|
||||
public void setState(int state) {
|
||||
this.data = (this.data & ~STATE_BITS) | state;
|
||||
}
|
||||
|
||||
public int getState() {
|
||||
return (int)(this.data & STATE_BITS);
|
||||
}
|
||||
|
||||
public void setHasWait() {
|
||||
this.data = this.data | WAITING_BIT;
|
||||
}
|
||||
|
||||
public boolean hasWait() {
|
||||
return (this.data & WAITING_BIT) == WAITING_BIT;
|
||||
}
|
||||
|
||||
public void reportActivity() {
|
||||
long now = EnvironmentEdgeManager.currentTimeMillis();
|
||||
this.data = (this.data & ALL_FLAG_BITS) | (now << 3);
|
||||
}
|
||||
|
||||
public boolean isExpired(long minRelevantTime) {
|
||||
return getActivityTime() < (minRelevantTime & (~0l >>> 3));
|
||||
}
|
||||
|
||||
private long getActivityTime() {
|
||||
return this.data >>> 3;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This implementation is not smart and just treats nonce group and nonce as random bits.
|
||||
*/
|
||||
// TODO: we could use pure byte arrays, but then we wouldn't be able to use hash map.
|
||||
private class NonceKey {
|
||||
private long group;
|
||||
private long nonce;
|
||||
|
||||
public NonceKey(long group, long nonce) {
|
||||
assert nonce != HConstants.NO_NONCE;
|
||||
this.group = group;
|
||||
this.nonce = nonce;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (obj == null || !(obj instanceof NonceKey)) return false;
|
||||
NonceKey nk = ((NonceKey)obj);
|
||||
return this.nonce == nk.nonce && this.group == nk.group;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return (int)((group >> 32) ^ group ^ (nonce >> 32) ^ nonce);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "[" + group + ":" + nonce + "]";
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Nonces.
|
||||
* Approximate overhead per nonce: 64 bytes from hashmap, 32 from two objects (k/v),
|
||||
* NK: 16 bytes (2 longs), OC: 8 bytes (1 long) - so, 120 bytes.
|
||||
* With 30min expiration time, 5k increments/appends per sec., we'd use approximately 1Gb,
|
||||
* which is a realistic worst case. If it's much worse, we could use some sort of memory
|
||||
* limit and cleanup.
|
||||
*/
|
||||
private ConcurrentHashMap<NonceKey, OperationContext> nonces =
|
||||
new ConcurrentHashMap<NonceKey, OperationContext>();
|
||||
|
||||
private int deleteNonceGracePeriod;
|
||||
|
||||
public ServerNonceManager(Configuration conf) {
|
||||
// Default - 30 minutes.
|
||||
deleteNonceGracePeriod = conf.getInt(HASH_NONCE_GRACE_PERIOD_KEY, 30 * 60 * 1000);
|
||||
if (deleteNonceGracePeriod < 60 * 1000) {
|
||||
LOG.warn("Nonce grace period " + deleteNonceGracePeriod
|
||||
+ " is less than a minute; might be too small to be useful");
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void setConflictWaitIterationMs(int conflictWaitIterationMs) {
|
||||
this.conflictWaitIterationMs = conflictWaitIterationMs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Starts the operation if operation with such nonce has not already succeeded. If the
|
||||
* operation is in progress, waits for it to end and checks whether it has succeeded.
|
||||
* @param group Nonce group.
|
||||
* @param nonce Nonce.
|
||||
* @param stoppable Stoppable that terminates waiting (if any) when the server is stopped.
|
||||
* @return true if the operation has not already succeeded and can proceed; false otherwise.
|
||||
*/
|
||||
public boolean startOperation(long group, long nonce, Stoppable stoppable)
|
||||
throws InterruptedException {
|
||||
if (nonce == HConstants.NO_NONCE) return true;
|
||||
NonceKey nk = new NonceKey(group, nonce);
|
||||
OperationContext ctx = new OperationContext();
|
||||
while (true) {
|
||||
OperationContext oldResult = nonces.putIfAbsent(nk, ctx);
|
||||
if (oldResult == null) return true;
|
||||
|
||||
// Collision with some operation - should be extremely rare.
|
||||
synchronized (oldResult) {
|
||||
int oldState = oldResult.getState();
|
||||
LOG.debug("Conflict detected by nonce: " + nk + ", " + oldResult);
|
||||
if (oldState != OperationContext.WAIT) {
|
||||
return oldState == OperationContext.PROCEED; // operation ended
|
||||
}
|
||||
oldResult.setHasWait();
|
||||
oldResult.wait(this.conflictWaitIterationMs); // operation is still active... wait and loop
|
||||
if (stoppable.isStopped()) {
|
||||
throw new InterruptedException("Server stopped");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Ends the operation started by startOperation.
|
||||
* @param group Nonce group.
|
||||
* @param nonce Nonce.
|
||||
* @param success Whether the operation has succeeded.
|
||||
*/
|
||||
public void endOperation(long group, long nonce, boolean success) {
|
||||
if (nonce == HConstants.NO_NONCE) return;
|
||||
NonceKey nk = new NonceKey(group, nonce);
|
||||
OperationContext newResult = nonces.get(nk);
|
||||
assert newResult != null;
|
||||
synchronized (newResult) {
|
||||
assert newResult.getState() == OperationContext.WAIT;
|
||||
// If we failed, other retries can proceed.
|
||||
newResult.setState(success ? OperationContext.DONT_PROCEED : OperationContext.PROCEED);
|
||||
if (success) {
|
||||
newResult.reportActivity(); // Set time to use for cleanup.
|
||||
} else {
|
||||
OperationContext val = nonces.remove(nk);
|
||||
assert val == newResult;
|
||||
}
|
||||
if (newResult.hasWait()) {
|
||||
LOG.debug("Conflict with running op ended: " + nk + ", " + newResult);
|
||||
newResult.notifyAll();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reports the operation from WAL during replay.
|
||||
* @param group Nonce group.
|
||||
* @param nonce Nonce.
|
||||
* @param writeTime Entry write time, used to ignore entries that are too old.
|
||||
*/
|
||||
public void reportOperationFromWal(long group, long nonce, long writeTime) {
|
||||
if (nonce == HConstants.NO_NONCE) return;
|
||||
// Give the write time some slack in case the clocks are not synchronized.
|
||||
long now = EnvironmentEdgeManager.currentTimeMillis();
|
||||
if (now > writeTime + (deleteNonceGracePeriod * 1.5)) return;
|
||||
OperationContext newResult = new OperationContext();
|
||||
newResult.setState(OperationContext.DONT_PROCEED);
|
||||
NonceKey nk = new NonceKey(group, nonce);
|
||||
OperationContext oldResult = nonces.putIfAbsent(nk, newResult);
|
||||
if (oldResult != null) {
|
||||
// Some schemes can have collisions (for example, expiring hashes), so just log it.
|
||||
// We have no idea about the semantics here, so this is the least of many evils.
|
||||
LOG.warn("Nonce collision during WAL recovery: " + nk
|
||||
+ ", " + oldResult + " with " + newResult);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a chore that is used to clean up old nonces.
|
||||
* @param stoppable Stoppable for the chore.
|
||||
* @return Chore; the chore is not started.
|
||||
*/
|
||||
public Chore createCleanupChore(Stoppable stoppable) {
|
||||
// By default, it will run every 6 minutes (30 / 5).
|
||||
return new Chore("nonceCleaner", deleteNonceGracePeriod / 5, stoppable) {
|
||||
@Override
|
||||
protected void chore() {
|
||||
cleanUpOldNonces();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private void cleanUpOldNonces() {
|
||||
long cutoff = EnvironmentEdgeManager.currentTimeMillis() - deleteNonceGracePeriod;
|
||||
for (Map.Entry<NonceKey, OperationContext> entry : nonces.entrySet()) {
|
||||
OperationContext oc = entry.getValue();
|
||||
if (!oc.isExpired(cutoff)) continue;
|
||||
synchronized (oc) {
|
||||
if (oc.getState() == OperationContext.WAIT || !oc.isExpired(cutoff)) continue;
|
||||
nonces.remove(entry.getKey());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -66,6 +66,8 @@ import org.apache.hadoop.util.StringUtils;
|
|||
import org.cloudera.htrace.Trace;
|
||||
import org.cloudera.htrace.TraceScope;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* HLog stores all the edits to the HStore. Its the hbase write-ahead-log
|
||||
* implementation.
|
||||
|
@ -898,21 +900,16 @@ class FSHLog implements HLog, Syncable {
|
|||
* @return New log key.
|
||||
*/
|
||||
protected HLogKey makeKey(byte[] encodedRegionName, TableName tableName, long seqnum,
|
||||
long now, List<UUID> clusterIds) {
|
||||
return new HLogKey(encodedRegionName, tableName, seqnum, now, clusterIds);
|
||||
long now, List<UUID> clusterIds, long nonceGroup, long nonce) {
|
||||
return new HLogKey(encodedRegionName, tableName, seqnum, now, clusterIds, nonceGroup, nonce);
|
||||
}
|
||||
|
||||
@Override
|
||||
@VisibleForTesting
|
||||
public void append(HRegionInfo info, TableName tableName, WALEdit edits,
|
||||
final long now, HTableDescriptor htd, AtomicLong sequenceId)
|
||||
throws IOException {
|
||||
append(info, tableName, edits, now, htd, true, sequenceId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void append(HRegionInfo info, TableName tableName, WALEdit edits, final long now,
|
||||
HTableDescriptor htd, boolean isInMemstore, AtomicLong sequenceId) throws IOException {
|
||||
append(info, tableName, edits, new ArrayList<UUID>(), now, htd, true, isInMemstore, sequenceId);
|
||||
final long now, HTableDescriptor htd, AtomicLong sequenceId) throws IOException {
|
||||
append(info, tableName, edits, new ArrayList<UUID>(), now, htd, true, true, sequenceId,
|
||||
HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -944,8 +941,8 @@ class FSHLog implements HLog, Syncable {
|
|||
*/
|
||||
@SuppressWarnings("deprecation")
|
||||
private long append(HRegionInfo info, TableName tableName, WALEdit edits, List<UUID> clusterIds,
|
||||
final long now, HTableDescriptor htd, boolean doSync, boolean isInMemstore, AtomicLong sequenceId)
|
||||
throws IOException {
|
||||
final long now, HTableDescriptor htd, boolean doSync, boolean isInMemstore,
|
||||
AtomicLong sequenceId, long nonceGroup, long nonce) throws IOException {
|
||||
if (edits.isEmpty()) return this.unflushedEntries.get();
|
||||
if (this.closed) {
|
||||
throw new IOException("Cannot append; log is closed");
|
||||
|
@ -966,7 +963,8 @@ class FSHLog implements HLog, Syncable {
|
|||
// actual name.
|
||||
byte [] encodedRegionName = info.getEncodedNameAsBytes();
|
||||
if (isInMemstore) this.oldestUnflushedSeqNums.putIfAbsent(encodedRegionName, seqNum);
|
||||
HLogKey logKey = makeKey(encodedRegionName, tableName, seqNum, now, clusterIds);
|
||||
HLogKey logKey = makeKey(
|
||||
encodedRegionName, tableName, seqNum, now, clusterIds, nonceGroup, nonce);
|
||||
doWrite(info, logKey, edits, htd);
|
||||
this.numEntries.incrementAndGet();
|
||||
txid = this.unflushedEntries.incrementAndGet();
|
||||
|
@ -975,6 +973,8 @@ class FSHLog implements HLog, Syncable {
|
|||
}
|
||||
this.latestSequenceNums.put(encodedRegionName, seqNum);
|
||||
}
|
||||
// TODO: note that only tests currently call append w/sync.
|
||||
// Therefore, this code here is not actually used by anything.
|
||||
// Sync if catalog region, and if not then check if that table supports
|
||||
// deferred log flushing
|
||||
if (doSync &&
|
||||
|
@ -991,9 +991,10 @@ class FSHLog implements HLog, Syncable {
|
|||
|
||||
@Override
|
||||
public long appendNoSync(HRegionInfo info, TableName tableName, WALEdit edits,
|
||||
List<UUID> clusterIds, final long now, HTableDescriptor htd, AtomicLong sequenceId)
|
||||
throws IOException {
|
||||
return append(info, tableName, edits, clusterIds, now, htd, false, true, sequenceId);
|
||||
List<UUID> clusterIds, final long now, HTableDescriptor htd, AtomicLong sequenceId,
|
||||
boolean isInMemstore, long nonceGroup, long nonce) throws IOException {
|
||||
return append(info, tableName, edits, clusterIds,
|
||||
now, htd, false, isInMemstore, sequenceId, nonceGroup, nonce);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -40,6 +40,8 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
|
||||
@InterfaceAudience.Private
|
||||
// TODO: Rename interface to WAL
|
||||
|
@ -260,24 +262,10 @@ public interface HLog {
|
|||
* except it causes a sync on the log
|
||||
* @param sequenceId of the region.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public void append(HRegionInfo info, TableName tableName, WALEdit edits,
|
||||
final long now, HTableDescriptor htd, AtomicLong sequenceId) throws IOException;
|
||||
|
||||
/**
|
||||
* Append a set of edits to the log. Log edits are keyed by (encoded)
|
||||
* regionName, row name, and log-sequence-id. The HLog is flushed after this
|
||||
* transaction is written to the log.
|
||||
* @param info
|
||||
* @param tableName
|
||||
* @param edits
|
||||
* @param now
|
||||
* @param htd
|
||||
* @param isInMemstore Whether the record is in memstore. False for system records.
|
||||
* @param sequenceId of the region.
|
||||
*/
|
||||
public void append(HRegionInfo info, TableName tableName, WALEdit edits, final long now,
|
||||
HTableDescriptor htd, boolean isInMemstore, AtomicLong sequenceId) throws IOException;
|
||||
|
||||
/**
|
||||
* Append a set of edits to the log. Log edits are keyed by (encoded) regionName, rowname, and
|
||||
* log-sequence-id. The HLog is not flushed after this transaction is written to the log.
|
||||
|
@ -291,8 +279,9 @@ public interface HLog {
|
|||
* @return txid of this transaction
|
||||
* @throws IOException
|
||||
*/
|
||||
public long appendNoSync(HRegionInfo info, TableName tableName, WALEdit edits,
|
||||
List<UUID> clusterIds, final long now, HTableDescriptor htd, AtomicLong sequenceId) throws IOException;
|
||||
long appendNoSync(HRegionInfo info, TableName tableName, WALEdit edits,
|
||||
List<UUID> clusterIds, final long now, HTableDescriptor htd, AtomicLong sequenceId,
|
||||
boolean isInMemstore, long nonceGroup, long nonce) throws IOException;
|
||||
|
||||
// TODO: Do we need all these versions of sync?
|
||||
void hsync() throws IOException;
|
||||
|
|
|
@ -45,6 +45,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.io.WritableComparable;
|
||||
import org.apache.hadoop.io.WritableUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.ByteString;
|
||||
import com.google.protobuf.ZeroCopyLiteralByteString;
|
||||
|
||||
|
@ -119,18 +120,23 @@ public class HLogKey implements WritableComparable<HLogKey> {
|
|||
|
||||
private NavigableMap<byte[], Integer> scopes;
|
||||
|
||||
private long nonceGroup = HConstants.NO_NONCE;
|
||||
private long nonce = HConstants.NO_NONCE;
|
||||
|
||||
private CompressionContext compressionContext;
|
||||
|
||||
public HLogKey() {
|
||||
init(null, null, 0L, HConstants.LATEST_TIMESTAMP,
|
||||
new ArrayList<UUID>());
|
||||
new ArrayList<UUID>(), HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public HLogKey(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
|
||||
final long now, UUID clusterId) {
|
||||
List<UUID> clusterIds = new ArrayList<UUID>();
|
||||
clusterIds.add(clusterId);
|
||||
init(encodedRegionName, tablename, logSeqNum, now, clusterIds);
|
||||
init(encodedRegionName, tablename, logSeqNum, now, clusterIds,
|
||||
HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -146,17 +152,19 @@ public class HLogKey implements WritableComparable<HLogKey> {
|
|||
* @param clusterIds the clusters that have consumed the change(used in Replication)
|
||||
*/
|
||||
public HLogKey(final byte [] encodedRegionName, final TableName tablename,
|
||||
long logSeqNum, final long now, List<UUID> clusterIds){
|
||||
init(encodedRegionName, tablename, logSeqNum, now, clusterIds);
|
||||
long logSeqNum, final long now, List<UUID> clusterIds, long nonceGroup, long nonce) {
|
||||
init(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce);
|
||||
}
|
||||
|
||||
protected void init(final byte [] encodedRegionName, final TableName tablename,
|
||||
long logSeqNum, final long now, List<UUID> clusterIds) {
|
||||
long logSeqNum, final long now, List<UUID> clusterIds, long nonceGroup, long nonce) {
|
||||
this.logSeqNum = logSeqNum;
|
||||
this.writeTime = now;
|
||||
this.clusterIds = clusterIds;
|
||||
this.encodedRegionName = encodedRegionName;
|
||||
this.tablename = tablename;
|
||||
this.nonceGroup = nonceGroup;
|
||||
this.nonce = nonce;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -192,6 +200,16 @@ public class HLogKey implements WritableComparable<HLogKey> {
|
|||
return scopes;
|
||||
}
|
||||
|
||||
/** @return The nonce group */
|
||||
public long getNonceGroup() {
|
||||
return nonceGroup;
|
||||
}
|
||||
|
||||
/** @return The nonce */
|
||||
public long getNonce() {
|
||||
return nonce;
|
||||
}
|
||||
|
||||
public void setScopes(NavigableMap<byte[], Integer> scopes) {
|
||||
this.scopes = scopes;
|
||||
}
|
||||
|
@ -435,6 +453,12 @@ public class HLogKey implements WritableComparable<HLogKey> {
|
|||
}
|
||||
builder.setLogSequenceNumber(this.logSeqNum);
|
||||
builder.setWriteTime(writeTime);
|
||||
if (this.nonce != HConstants.NO_NONCE) {
|
||||
builder.setNonce(nonce);
|
||||
}
|
||||
if (this.nonceGroup != HConstants.NO_NONCE) {
|
||||
builder.setNonceGroup(nonceGroup);
|
||||
}
|
||||
HBaseProtos.UUID.Builder uuidBuilder = HBaseProtos.UUID.newBuilder();
|
||||
for (UUID clusterId : clusterIds) {
|
||||
uuidBuilder.setLeastSigBits(clusterId.getLeastSignificantBits());
|
||||
|
@ -474,6 +498,12 @@ public class HLogKey implements WritableComparable<HLogKey> {
|
|||
for (HBaseProtos.UUID clusterId : walKey.getClusterIdsList()) {
|
||||
clusterIds.add(new UUID(clusterId.getMostSigBits(), clusterId.getLeastSigBits()));
|
||||
}
|
||||
if (walKey.hasNonceGroup()) {
|
||||
this.nonceGroup = walKey.getNonceGroup();
|
||||
}
|
||||
if (walKey.hasNonce()) {
|
||||
this.nonce = walKey.getNonce();
|
||||
}
|
||||
this.scopes = null;
|
||||
if (walKey.getScopesCount() > 0) {
|
||||
this.scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
|
||||
|
|
|
@ -1836,6 +1836,21 @@ public class HLogSplitter {
|
|||
}
|
||||
}
|
||||
|
||||
/** A struct used by getMutationsFromWALEntry */
|
||||
public static class MutationReplay {
|
||||
public MutationReplay(MutationType type, Mutation mutation, long nonceGroup, long nonce) {
|
||||
this.type = type;
|
||||
this.mutation = mutation;
|
||||
this.nonceGroup = nonceGroup;
|
||||
this.nonce = nonce;
|
||||
}
|
||||
|
||||
public final MutationType type;
|
||||
public final Mutation mutation;
|
||||
public final long nonceGroup;
|
||||
public final long nonce;
|
||||
}
|
||||
|
||||
/**
|
||||
* This function is used to construct mutations from a WALEntry. It also reconstructs HLogKey &
|
||||
* WALEdit from the passed in WALEntry
|
||||
|
@ -1846,16 +1861,16 @@ public class HLogSplitter {
|
|||
* @return list of Pair<MutationType, Mutation> to be replayed
|
||||
* @throws IOException
|
||||
*/
|
||||
public static List<Pair<MutationType, Mutation>> getMutationsFromWALEntry(WALEntry entry,
|
||||
public static List<MutationReplay> getMutationsFromWALEntry(WALEntry entry,
|
||||
CellScanner cells, Pair<HLogKey, WALEdit> logEntry) throws IOException {
|
||||
|
||||
if (entry == null) {
|
||||
// return an empty array
|
||||
return new ArrayList<Pair<MutationType, Mutation>>();
|
||||
return new ArrayList<MutationReplay>();
|
||||
}
|
||||
|
||||
int count = entry.getAssociatedCellCount();
|
||||
List<Pair<MutationType, Mutation>> mutations = new ArrayList<Pair<MutationType, Mutation>>();
|
||||
List<MutationReplay> mutations = new ArrayList<MutationReplay>();
|
||||
Cell previousCell = null;
|
||||
Mutation m = null;
|
||||
HLogKey key = null;
|
||||
|
@ -1877,10 +1892,16 @@ public class HLogSplitter {
|
|||
// Create new mutation
|
||||
if (CellUtil.isDelete(cell)) {
|
||||
m = new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
|
||||
mutations.add(new Pair<MutationType, Mutation>(MutationType.DELETE, m));
|
||||
// Deletes don't have nonces.
|
||||
mutations.add(new MutationReplay(
|
||||
MutationType.DELETE, m, HConstants.NO_NONCE, HConstants.NO_NONCE));
|
||||
} else {
|
||||
m = new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
|
||||
mutations.add(new Pair<MutationType, Mutation>(MutationType.PUT, m));
|
||||
// Puts might come from increment or append, thus we need nonces.
|
||||
long nonceGroup = entry.getKey().hasNonceGroup()
|
||||
? entry.getKey().getNonceGroup() : HConstants.NO_NONCE;
|
||||
long nonce = entry.getKey().hasNonce() ? entry.getKey().getNonce() : HConstants.NO_NONCE;
|
||||
mutations.add(new MutationReplay(MutationType.PUT, m, nonceGroup, nonce));
|
||||
}
|
||||
}
|
||||
if (CellUtil.isDelete(cell)) {
|
||||
|
@ -1900,7 +1921,7 @@ public class HLogSplitter {
|
|||
}
|
||||
key = new HLogKey(walKey.getEncodedRegionName().toByteArray(), TableName.valueOf(walKey
|
||||
.getTableName().toByteArray()), walKey.getLogSequenceNumber(), walKey.getWriteTime(),
|
||||
clusterIds);
|
||||
clusterIds, walKey.getNonceGroup(), walKey.getNonce());
|
||||
logEntry.setFirst(key);
|
||||
logEntry.setSecond(val);
|
||||
}
|
||||
|
|
|
@ -20,8 +20,10 @@
|
|||
package org.apache.hadoop.hbase.regionserver.wal;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.TreeSet;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
@ -262,8 +264,12 @@ public class HLogUtil {
|
|||
public static void writeCompactionMarker(HLog log, HTableDescriptor htd, HRegionInfo info,
|
||||
final CompactionDescriptor c, AtomicLong sequenceId) throws IOException {
|
||||
WALEdit e = WALEdit.createCompaction(c);
|
||||
log.append(info, TableName.valueOf(c.getTableName().toByteArray()), e,
|
||||
EnvironmentEdgeManager.currentTimeMillis(), htd, false, sequenceId);
|
||||
long now = EnvironmentEdgeManager.currentTimeMillis();
|
||||
TableName tn = TableName.valueOf(c.getTableName().toByteArray());
|
||||
long txid = log.appendNoSync(info, tn, e, new ArrayList<UUID>(), now, htd, sequenceId,
|
||||
false, HConstants.NO_NONCE, HConstants.NO_NONCE);
|
||||
log.sync(txid);
|
||||
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Appended compaction marker " + TextFormat.shortDebugString(c));
|
||||
}
|
||||
|
|
|
@ -171,8 +171,8 @@ class SnapshotLogSplitter implements Closeable {
|
|||
}
|
||||
|
||||
// Append Entry
|
||||
key = new HLogKey(newRegionName, tableName,
|
||||
key.getLogSeqNum(), key.getWriteTime(), key.getClusterIds());
|
||||
key = new HLogKey(newRegionName, tableName, key.getLogSeqNum(), key.getWriteTime(),
|
||||
key.getClusterIds(), key.getNonceGroup(), key.getNonce());
|
||||
writer.append(new HLog.Entry(key, entry.getEdit()));
|
||||
}
|
||||
} catch (IOException e) {
|
||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
|
|||
import org.apache.hadoop.hbase.regionserver.Leases;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.regionserver.ServerNonceManager;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
|
@ -62,7 +63,7 @@ class MockRegionServerServices implements RegionServerServices {
|
|||
MockRegionServerServices(ZooKeeperWatcher zkw) {
|
||||
this.zkw = zkw;
|
||||
}
|
||||
|
||||
|
||||
MockRegionServerServices(ZooKeeperWatcher zkw, ServerName serverName) {
|
||||
this.zkw = zkw;
|
||||
this.serverName = serverName;
|
||||
|
@ -192,7 +193,6 @@ class MockRegionServerServices implements RegionServerServices {
|
|||
|
||||
@Override
|
||||
public HLog getWAL(HRegionInfo regionInfo) throws IOException {
|
||||
// TODO Auto-generated method stub
|
||||
return null;
|
||||
}
|
||||
|
||||
|
@ -221,4 +221,10 @@ class MockRegionServerServices implements RegionServerServices {
|
|||
public int getPriority(RPCProtos.RequestHeader header, Message param) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServerNonceManager getNonceManager() {
|
||||
// TODO Auto-generated method stub
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,6 +27,7 @@ import java.io.IOException;
|
|||
import java.lang.reflect.Field;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -37,11 +38,13 @@ import org.apache.hadoop.hbase.CellUtil;
|
|||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.MediumTests;
|
||||
import org.apache.hadoop.hbase.Waiter;
|
||||
import org.apache.hadoop.hbase.exceptions.OperationConflictException;
|
||||
import org.apache.hadoop.hbase.ipc.RpcClient;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServer;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.JVMClusterUtil;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.log4j.Level;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
|
@ -481,6 +484,96 @@ public class TestMultiParallel {
|
|||
table.close();
|
||||
}
|
||||
|
||||
@Test(timeout=300000)
|
||||
public void testNonceCollision() throws Exception {
|
||||
LOG.info("test=testNonceCollision");
|
||||
HTable table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
|
||||
Put put = new Put(ONE_ROW);
|
||||
put.add(BYTES_FAMILY, QUALIFIER, Bytes.toBytes(0L));
|
||||
|
||||
// Replace nonce manager with the one that returns each nonce twice.
|
||||
NonceGenerator cnm = new PerClientRandomNonceGenerator() {
|
||||
long lastNonce = -1;
|
||||
@Override
|
||||
public synchronized long newNonce() {
|
||||
long nonce = 0;
|
||||
if (lastNonce == -1) {
|
||||
lastNonce = nonce = super.newNonce();
|
||||
} else {
|
||||
nonce = lastNonce;
|
||||
lastNonce = -1L;
|
||||
}
|
||||
return nonce;
|
||||
}
|
||||
};
|
||||
NonceGenerator oldCnm =
|
||||
HConnectionManager.injectNonceGeneratorForTesting(table.getConnection(), cnm);
|
||||
|
||||
// First test sequential requests.
|
||||
try {
|
||||
Increment inc = new Increment(ONE_ROW);
|
||||
inc.addColumn(BYTES_FAMILY, QUALIFIER, 1L);
|
||||
table.increment(inc);
|
||||
inc = new Increment(ONE_ROW);
|
||||
inc.addColumn(BYTES_FAMILY, QUALIFIER, 1L);
|
||||
try {
|
||||
table.increment(inc);
|
||||
fail("Should have thrown an exception");
|
||||
} catch (OperationConflictException ex) {
|
||||
}
|
||||
Get get = new Get(ONE_ROW);
|
||||
get.addColumn(BYTES_FAMILY, QUALIFIER);
|
||||
Result result = table.get(get);
|
||||
validateResult(result, QUALIFIER, Bytes.toBytes(1L));
|
||||
|
||||
// Now run a bunch of requests in parallel, exactly half should succeed.
|
||||
int numRequests = 40;
|
||||
final CountDownLatch startedLatch = new CountDownLatch(numRequests);
|
||||
final CountDownLatch startLatch = new CountDownLatch(1);
|
||||
final CountDownLatch doneLatch = new CountDownLatch(numRequests);
|
||||
for (int i = 0; i < numRequests; ++i) {
|
||||
Runnable r = new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
HTable table = null;
|
||||
try {
|
||||
table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
|
||||
} catch (IOException e) {
|
||||
fail("Not expected");
|
||||
}
|
||||
Increment inc = new Increment(ONE_ROW);
|
||||
inc.addColumn(BYTES_FAMILY, QUALIFIER, 1L);
|
||||
startedLatch.countDown();
|
||||
try {
|
||||
startLatch.await();
|
||||
} catch (InterruptedException e) {
|
||||
fail("Not expected");
|
||||
}
|
||||
try {
|
||||
table.increment(inc);
|
||||
} catch (OperationConflictException ex) { // Some threads are expected to fail.
|
||||
} catch (IOException ioEx) {
|
||||
fail("Not expected");
|
||||
}
|
||||
doneLatch.countDown();
|
||||
}
|
||||
};
|
||||
Threads.setDaemonThreadRunning(new Thread(r));
|
||||
}
|
||||
startedLatch.await(); // Wait until all threads are ready...
|
||||
startLatch.countDown(); // ...and unleash the herd!
|
||||
doneLatch.await();
|
||||
// Now verify
|
||||
get = new Get(ONE_ROW);
|
||||
get.addColumn(BYTES_FAMILY, QUALIFIER);
|
||||
result = table.get(get);
|
||||
validateResult(result, QUALIFIER, Bytes.toBytes((numRequests / 2) + 1L));
|
||||
table.close();
|
||||
} finally {
|
||||
HConnectionManager.injectNonceGeneratorForTesting(table.getConnection(), oldCnm);
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=300000)
|
||||
public void testBatchWithMixedActions() throws Exception {
|
||||
LOG.info("test=testBatchWithMixedActions");
|
||||
|
@ -558,10 +651,13 @@ public class TestMultiParallel {
|
|||
}
|
||||
|
||||
private void validateResult(Object r1, byte[] qual, byte[] val) {
|
||||
// TODO provide nice assert here or something.
|
||||
Result r = (Result)r1;
|
||||
Assert.assertTrue(r.containsColumn(BYTES_FAMILY, qual));
|
||||
Assert.assertEquals(0, Bytes.compareTo(val, r.getValue(BYTES_FAMILY, qual)));
|
||||
byte[] value = r.getValue(BYTES_FAMILY, qual);
|
||||
if (0 != Bytes.compareTo(val, value)) {
|
||||
fail("Expected [" + Bytes.toStringBinary(val)
|
||||
+ "] but got [" + Bytes.toStringBinary(value) + "]");
|
||||
}
|
||||
}
|
||||
|
||||
private List<Row> constructPutRequests() {
|
||||
|
|
|
@ -91,6 +91,7 @@ import org.apache.hadoop.hbase.regionserver.HRegion;
|
|||
import org.apache.hadoop.hbase.regionserver.Leases;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.regionserver.ServerNonceManager;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.HLog;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
|
||||
|
@ -554,4 +555,9 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
|
|||
UpdateFavoredNodesRequest request) throws ServiceException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ServerNonceManager getNonceManager() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_acquired;
|
|||
import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_done;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_err;
|
||||
import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_resigned;
|
||||
import static org.junit.Assert.*;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
@ -35,6 +36,7 @@ import java.util.ArrayList;
|
|||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.Set;
|
||||
|
@ -68,9 +70,14 @@ import org.apache.hadoop.hbase.ServerName;
|
|||
import org.apache.hadoop.hbase.SplitLogCounters;
|
||||
import org.apache.hadoop.hbase.Waiter;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.HConnectionManager;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Increment;
|
||||
import org.apache.hadoop.hbase.client.NonceGenerator;
|
||||
import org.apache.hadoop.hbase.client.PerClientRandomNonceGenerator;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
|
||||
import org.apache.hadoop.hbase.exceptions.OperationConflictException;
|
||||
import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException;
|
||||
import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
|
@ -172,7 +179,7 @@ public class TestDistributedLogSplitting {
|
|||
try {
|
||||
if (TEST_UTIL.getHBaseCluster() != null) {
|
||||
for (MasterThread mt : TEST_UTIL.getHBaseCluster().getLiveMasterThreads()) {
|
||||
mt.getMaster().abort("closing...", new Exception("Trace info"));
|
||||
mt.getMaster().abort("closing...", null);
|
||||
}
|
||||
}
|
||||
TEST_UTIL.shutdownMiniHBaseCluster();
|
||||
|
@ -273,6 +280,80 @@ public class TestDistributedLogSplitting {
|
|||
zkw.close();
|
||||
}
|
||||
|
||||
private static class NonceGeneratorWithDups extends PerClientRandomNonceGenerator {
|
||||
private boolean isDups = false;
|
||||
private LinkedList<Long> nonces = new LinkedList<Long>();
|
||||
|
||||
public void startDups() {
|
||||
isDups = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long newNonce() {
|
||||
long nonce = isDups ? nonces.removeFirst() : super.newNonce();
|
||||
if (!isDups) {
|
||||
nonces.add(nonce);
|
||||
}
|
||||
return nonce;
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
public void testNonceRecovery() throws Exception {
|
||||
LOG.info("testNonceRecovery");
|
||||
final String TABLE_NAME = "table";
|
||||
final String FAMILY_NAME = "family";
|
||||
final int NUM_REGIONS_TO_CREATE = 40;
|
||||
|
||||
conf.setLong("hbase.regionserver.hlog.blocksize", 100*1024);
|
||||
conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, true);
|
||||
startCluster(NUM_RS);
|
||||
master.balanceSwitch(false);
|
||||
|
||||
final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
|
||||
HTable ht = installTable(zkw, TABLE_NAME, FAMILY_NAME, NUM_REGIONS_TO_CREATE);
|
||||
NonceGeneratorWithDups ng = new NonceGeneratorWithDups();
|
||||
NonceGenerator oldNg =
|
||||
HConnectionManager.injectNonceGeneratorForTesting(ht.getConnection(), ng);
|
||||
|
||||
try {
|
||||
List<Increment> reqs = new ArrayList<Increment>();
|
||||
for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
|
||||
HRegionServer hrs = rst.getRegionServer();
|
||||
List<HRegionInfo> hris = ProtobufUtil.getOnlineRegions(hrs);
|
||||
for (HRegionInfo hri : hris) {
|
||||
if (TABLE_NAME.equalsIgnoreCase(hri.getTable().getNameAsString())) {
|
||||
byte[] key = hri.getStartKey();
|
||||
if (key == null || key.length == 0) {
|
||||
key = Bytes.copy(hri.getEndKey());
|
||||
--(key[key.length - 1]);
|
||||
}
|
||||
Increment incr = new Increment(key);
|
||||
incr.addColumn(Bytes.toBytes(FAMILY_NAME), Bytes.toBytes("q"), 1);
|
||||
ht.increment(incr);
|
||||
reqs.add(incr);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
HRegionServer hrs = findRSToKill(false, "table");
|
||||
abortRSAndWaitForRecovery(hrs, zkw, NUM_REGIONS_TO_CREATE);
|
||||
ng.startDups();
|
||||
for (Increment incr : reqs) {
|
||||
try {
|
||||
ht.increment(incr);
|
||||
fail("should have thrown");
|
||||
} catch (OperationConflictException ope) {
|
||||
LOG.debug("Caught as expected: " + ope.getMessage());
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
HConnectionManager.injectNonceGeneratorForTesting(ht.getConnection(), oldNg);
|
||||
ht.close();
|
||||
zkw.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
public void testLogReplayWithMetaRSDown() throws Exception {
|
||||
LOG.info("testRecoveredEditsReplayWithMetaRSDown");
|
||||
|
|
|
@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
|
@ -220,7 +221,7 @@ public class TestProtobufUtil {
|
|||
|
||||
Increment increment = ProtobufUtil.toIncrement(proto, null);
|
||||
assertEquals(mutateBuilder.build(),
|
||||
ProtobufUtil.toMutation(increment, MutationProto.newBuilder()));
|
||||
ProtobufUtil.toMutation(increment, MutationProto.newBuilder(), HConstants.NO_NONCE));
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -33,9 +33,7 @@ import static org.junit.Assert.assertNotNull;
|
|||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.mockito.Matchers.any;
|
||||
import static org.mockito.Matchers.anyLong;
|
||||
import static org.mockito.Matchers.eq;
|
||||
import static org.mockito.Matchers.*;
|
||||
import static org.mockito.Mockito.never;
|
||||
import static org.mockito.Mockito.spy;
|
||||
import static org.mockito.Mockito.times;
|
||||
|
@ -3800,10 +3798,11 @@ public class TestHRegion {
|
|||
put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1"));
|
||||
put.setDurability(mutationDurability);
|
||||
region.put(put);
|
||||
|
||||
//verify append called or not
|
||||
verify(log, expectAppend ? times(1) : never())
|
||||
.appendNoSync((HRegionInfo)any(), eq(tableName), (WALEdit)any(), (List<UUID>)any(),
|
||||
anyLong(), (HTableDescriptor)any(), (AtomicLong)any());
|
||||
anyLong(), (HTableDescriptor)any(), (AtomicLong)any(), anyBoolean(), anyLong(), anyLong());
|
||||
|
||||
// verify sync called or not
|
||||
if (expectSync || expectSyncFromLogSyncer) {
|
||||
|
|
|
@ -0,0 +1,281 @@
|
|||
/*
|
||||
*
|
||||
* 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.regionserver;
|
||||
|
||||
import static org.apache.hadoop.hbase.HConstants.NO_NONCE;
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.Chore;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.SmallTests;
|
||||
import org.apache.hadoop.hbase.Stoppable;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.mockito.Mockito;
|
||||
import org.mockito.invocation.InvocationOnMock;
|
||||
import org.mockito.stubbing.Answer;
|
||||
|
||||
@Category(SmallTests.class)
|
||||
public class TestServerNonceManager {
|
||||
|
||||
@Test
|
||||
public void testNormalStartEnd() throws Exception {
|
||||
final long[] numbers = new long[] { NO_NONCE, 1, 2, Long.MAX_VALUE, Long.MIN_VALUE };
|
||||
ServerNonceManager nm = createManager();
|
||||
for (int i = 0; i < numbers.length; ++i) {
|
||||
for (int j = 0; j < numbers.length; ++j) {
|
||||
assertTrue(nm.startOperation(numbers[i], numbers[j], createStoppable()));
|
||||
}
|
||||
}
|
||||
// Should be able to start operation the second time w/o nonces.
|
||||
for (int i = 0; i < numbers.length; ++i) {
|
||||
assertTrue(nm.startOperation(numbers[i], NO_NONCE, createStoppable()));
|
||||
}
|
||||
// Fail all operations - should be able to restart.
|
||||
for (int i = 0; i < numbers.length; ++i) {
|
||||
for (int j = 0; j < numbers.length; ++j) {
|
||||
nm.endOperation(numbers[i], numbers[j], false);
|
||||
assertTrue(nm.startOperation(numbers[i], numbers[j], createStoppable()));
|
||||
}
|
||||
}
|
||||
// Succeed all operations - should not be able to restart, except for NO_NONCE.
|
||||
for (int i = 0; i < numbers.length; ++i) {
|
||||
for (int j = 0; j < numbers.length; ++j) {
|
||||
nm.endOperation(numbers[i], numbers[j], true);
|
||||
assertEquals(numbers[j] == NO_NONCE,
|
||||
nm.startOperation(numbers[i], numbers[j], createStoppable()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNoEndWithoutStart() {
|
||||
ServerNonceManager nm = createManager();
|
||||
try {
|
||||
nm.endOperation(NO_NONCE, 1, true);
|
||||
fail("Should have thrown");
|
||||
} catch (AssertionError err) {}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCleanup() throws Exception {
|
||||
ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
|
||||
EnvironmentEdgeManager.injectEdge(edge);
|
||||
try {
|
||||
ServerNonceManager nm = createManager(6);
|
||||
Chore cleanup = nm.createCleanupChore(Mockito.mock(Stoppable.class));
|
||||
edge.setValue(1);
|
||||
assertTrue(nm.startOperation(NO_NONCE, 1, createStoppable()));
|
||||
assertTrue(nm.startOperation(NO_NONCE, 2, createStoppable()));
|
||||
assertTrue(nm.startOperation(NO_NONCE, 3, createStoppable()));
|
||||
edge.setValue(2);
|
||||
nm.endOperation(NO_NONCE, 1, true);
|
||||
edge.setValue(4);
|
||||
nm.endOperation(NO_NONCE, 2, true);
|
||||
edge.setValue(9);
|
||||
cleanup.choreForTesting();
|
||||
// Nonce 1 has been cleaned up.
|
||||
assertTrue(nm.startOperation(NO_NONCE, 1, createStoppable()));
|
||||
// Nonce 2 has not been cleaned up.
|
||||
assertFalse(nm.startOperation(NO_NONCE, 2, createStoppable()));
|
||||
// Nonce 3 was active and active ops should never be cleaned up; try to end and start.
|
||||
nm.endOperation(NO_NONCE, 3, false);
|
||||
assertTrue(nm.startOperation(NO_NONCE, 3, createStoppable()));
|
||||
edge.setValue(11);
|
||||
cleanup.choreForTesting();
|
||||
// Now, nonce 2 has been cleaned up.
|
||||
assertTrue(nm.startOperation(NO_NONCE, 2, createStoppable()));
|
||||
} finally {
|
||||
EnvironmentEdgeManager.reset();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWalNonces() throws Exception {
|
||||
ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
|
||||
EnvironmentEdgeManager.injectEdge(edge);
|
||||
try {
|
||||
ServerNonceManager nm = createManager(6);
|
||||
Chore cleanup = nm.createCleanupChore(Mockito.mock(Stoppable.class));
|
||||
// Add nonces from WAL, including dups.
|
||||
edge.setValue(12);
|
||||
nm.reportOperationFromWal(NO_NONCE, 1, 8);
|
||||
nm.reportOperationFromWal(NO_NONCE, 2, 2);
|
||||
nm.reportOperationFromWal(NO_NONCE, 3, 5);
|
||||
nm.reportOperationFromWal(NO_NONCE, 3, 6);
|
||||
// WAL nonces should prevent cross-server conflicts.
|
||||
assertFalse(nm.startOperation(NO_NONCE, 1, createStoppable()));
|
||||
// Make sure we ignore very old nonces, but not borderline old nonces.
|
||||
assertTrue(nm.startOperation(NO_NONCE, 2, createStoppable()));
|
||||
assertFalse(nm.startOperation(NO_NONCE, 3, createStoppable()));
|
||||
// Make sure grace period is counted from recovery time.
|
||||
edge.setValue(17);
|
||||
cleanup.choreForTesting();
|
||||
assertFalse(nm.startOperation(NO_NONCE, 1, createStoppable()));
|
||||
assertFalse(nm.startOperation(NO_NONCE, 3, createStoppable()));
|
||||
edge.setValue(19);
|
||||
cleanup.choreForTesting();
|
||||
assertTrue(nm.startOperation(NO_NONCE, 1, createStoppable()));
|
||||
assertTrue(nm.startOperation(NO_NONCE, 3, createStoppable()));
|
||||
} finally {
|
||||
EnvironmentEdgeManager.reset();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConcurrentAttempts() throws Exception {
|
||||
final ServerNonceManager nm = createManager();
|
||||
|
||||
nm.startOperation(NO_NONCE, 1, createStoppable());
|
||||
TestRunnable tr = new TestRunnable(nm, 1, false, createStoppable());
|
||||
Thread t = tr.start();
|
||||
waitForThreadToBlockOrExit(t);
|
||||
nm.endOperation(NO_NONCE, 1, true); // operation succeeded
|
||||
t.join(); // thread must now unblock and not proceed (result checked inside).
|
||||
tr.propagateError();
|
||||
|
||||
nm.startOperation(NO_NONCE, 2, createStoppable());
|
||||
tr = new TestRunnable(nm, 2, true, createStoppable());
|
||||
t = tr.start();
|
||||
waitForThreadToBlockOrExit(t);
|
||||
nm.endOperation(NO_NONCE, 2, false);
|
||||
t.join(); // thread must now unblock and allow us to proceed (result checked inside).
|
||||
tr.propagateError();
|
||||
nm.endOperation(NO_NONCE, 2, true); // that is to say we should be able to end operation
|
||||
|
||||
nm.startOperation(NO_NONCE, 3, createStoppable());
|
||||
tr = new TestRunnable(nm, 4, true, createStoppable());
|
||||
tr.start().join(); // nonce 3 must have no bearing on nonce 4
|
||||
tr.propagateError();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStopWaiting() throws Exception {
|
||||
final ServerNonceManager nm = createManager();
|
||||
nm.setConflictWaitIterationMs(1);
|
||||
Stoppable stoppingStoppable = createStoppable();
|
||||
Mockito.when(stoppingStoppable.isStopped()).thenAnswer(new Answer<Boolean>() {
|
||||
AtomicInteger answer = new AtomicInteger(3);
|
||||
@Override
|
||||
public Boolean answer(InvocationOnMock invocation) throws Throwable {
|
||||
return 0 < answer.decrementAndGet();
|
||||
}
|
||||
});
|
||||
|
||||
nm.startOperation(NO_NONCE, 1, createStoppable());
|
||||
TestRunnable tr = new TestRunnable(nm, 1, null, stoppingStoppable);
|
||||
Thread t = tr.start();
|
||||
waitForThreadToBlockOrExit(t);
|
||||
// thread must eventually throw
|
||||
t.join();
|
||||
tr.propagateError();
|
||||
}
|
||||
|
||||
private void waitForThreadToBlockOrExit(Thread t) throws InterruptedException {
|
||||
for (int i = 9; i >= 0; --i) {
|
||||
if (t.getState() == Thread.State.TIMED_WAITING || t.getState() == Thread.State.WAITING
|
||||
|| t.getState() == Thread.State.BLOCKED || t.getState() == Thread.State.TERMINATED) {
|
||||
return;
|
||||
}
|
||||
if (i > 0) Thread.sleep(300);
|
||||
}
|
||||
// Thread didn't block in 3 seconds. What is it doing? Continue the test, we'd rather
|
||||
// have a very strange false positive then false negative due to timing.
|
||||
}
|
||||
|
||||
private static class TestRunnable implements Runnable {
|
||||
public final CountDownLatch startedLatch = new CountDownLatch(1); // It's the final countdown!
|
||||
|
||||
private final ServerNonceManager nm;
|
||||
private final long nonce;
|
||||
private final Boolean expected;
|
||||
private final Stoppable stoppable;
|
||||
|
||||
private Throwable throwable = null;
|
||||
|
||||
public TestRunnable(ServerNonceManager nm, long nonce, Boolean expected, Stoppable stoppable) {
|
||||
this.nm = nm;
|
||||
this.nonce = nonce;
|
||||
this.expected = expected;
|
||||
this.stoppable = stoppable;
|
||||
}
|
||||
|
||||
public void propagateError() throws Exception {
|
||||
if (throwable == null) return;
|
||||
throw new Exception(throwable);
|
||||
}
|
||||
|
||||
public Thread start() {
|
||||
Thread t = new Thread(this);
|
||||
t = Threads.setDaemonThreadRunning(t);
|
||||
try {
|
||||
startedLatch.await();
|
||||
} catch (InterruptedException e) {
|
||||
fail("Unexpected");
|
||||
}
|
||||
return t;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
startedLatch.countDown();
|
||||
boolean shouldThrow = expected == null;
|
||||
boolean hasThrown = true;
|
||||
try {
|
||||
boolean result = nm.startOperation(NO_NONCE, nonce, stoppable);
|
||||
hasThrown = false;
|
||||
if (!shouldThrow) {
|
||||
assertEquals(expected.booleanValue(), result);
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
if (!shouldThrow) {
|
||||
throwable = t;
|
||||
}
|
||||
}
|
||||
if (shouldThrow && !hasThrown) {
|
||||
throwable = new AssertionError("Should have thrown");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private Stoppable createStoppable() {
|
||||
Stoppable s = Mockito.mock(Stoppable.class);
|
||||
Mockito.when(s.isStopped()).thenReturn(false);
|
||||
return s;
|
||||
}
|
||||
|
||||
private ServerNonceManager createManager() {
|
||||
return createManager(null);
|
||||
}
|
||||
|
||||
private ServerNonceManager createManager(Integer gracePeriod) {
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
if (gracePeriod != null) {
|
||||
conf.setInt(ServerNonceManager.HASH_NONCE_GRACE_PERIOD_KEY, gracePeriod.intValue());
|
||||
}
|
||||
return new ServerNonceManager(conf);
|
||||
}
|
||||
}
|
|
@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
|
@ -104,6 +105,8 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool
|
|||
byte[] value = new byte[valueSize];
|
||||
Random rand = new Random(Thread.currentThread().getId());
|
||||
HLog hlog = region.getLog();
|
||||
ArrayList<UUID> clusters = new ArrayList<UUID>();
|
||||
long nonce = HConstants.NO_NONCE;
|
||||
|
||||
try {
|
||||
long startTime = System.currentTimeMillis();
|
||||
|
@ -114,8 +117,8 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool
|
|||
addFamilyMapToWALEdit(put.getFamilyCellMap(), walEdit);
|
||||
HRegionInfo hri = region.getRegionInfo();
|
||||
if (this.noSync) {
|
||||
hlog.appendNoSync(hri, hri.getTable(), walEdit, new ArrayList<UUID>(), now, htd,
|
||||
region.getSequenceId());
|
||||
hlog.appendNoSync(hri, hri.getTable(), walEdit, clusters, now, htd,
|
||||
region.getSequenceId(), true, nonce, nonce);
|
||||
} else {
|
||||
hlog.append(hri, hri.getTable(), walEdit, now, htd, region.getSequenceId());
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue