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:
sershe 2013-11-15 04:36:30 +00:00
parent db4f10c208
commit 6119df9ccb
45 changed files with 2579 additions and 284 deletions

View File

@ -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;
}
}

View File

@ -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);
}
}

View File

@ -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];
}
}

View File

@ -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();
}

View File

@ -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

View File

@ -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,12 +1017,14 @@ 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);
getLocation().getRegionInfo().getRegionName(), increment, nonceGroup, nonce);
PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
rpcController.setPriority(getTableName());
MutateResponse response = getStub().mutate(rpcController, request);
@ -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);

View File

@ -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;
}
}

View File

@ -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);

View File

@ -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();
}

View File

@ -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;
}
}

View File

@ -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);
}
}

View File

@ -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();
}

View File

@ -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 {

View File

@ -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.
}

View File

@ -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);

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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 {

View File

@ -30,6 +30,8 @@ message MultiRowMutationProcessorResponse{
message MutateRowsRequest {
repeated MutationProto mutation_request = 1;
optional uint64 nonce_group = 2;
optional uint64 nonce = 3;
}
message MutateRowsResponse {

View File

@ -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 {

View File

@ -43,11 +43,16 @@ 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;

View File

@ -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
}
}

View File

@ -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());

View File

@ -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;
@ -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);
}

View File

@ -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());

View File

@ -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,21 +1913,82 @@ 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;
}
}
/**
* Perform a batch of mutations.
* It supports only Put and Delete mutations and will ignore other types passed.
* @param mutations the list of mutations
* @return an array of OperationStatus which internally contains the
* OperationStatusCode and the exceptionMessage if any.
* @throws IOException
*/
public OperationStatus[] batchMutate(Mutation[] mutations) throws IOException {
return batchMutate(mutations, false);
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;
}
}
/**
@ -1935,21 +1999,48 @@ public class HRegion implements HeapSize { // , Writable{
* OperationStatusCode and the exceptionMessage if any.
* @throws IOException
*/
OperationStatus[] batchMutate(Mutation[] mutations, boolean isReplay)
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, 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 {
BatchOperationInProgress<Mutation> batchOp =
new BatchOperationInProgress<Mutation>(mutations);
return batchMutate(new ReplayBatch(mutations));
}
/**
* Perform a batch of mutations.
* It supports only Put and Delete mutations and will ignore other types passed.
* @param mutations the list of mutations
* @return an array of OperationStatus which internally contains the
* OperationStatusCode and the exceptionMessage if any.
* @throws IOException
*/
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 {
@ -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());
}

View File

@ -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) {

View File

@ -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();
}

View File

@ -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());
}
}
}
}

View File

@ -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);
}
/**

View File

@ -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;

View File

@ -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);

View File

@ -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);
}

View File

@ -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));
}

View File

@ -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) {

View File

@ -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;
@ -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;
}
}

View File

@ -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() {

View File

@ -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;
}
}

View File

@ -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");

View File

@ -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));
}
/**

View File

@ -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) {

View File

@ -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);
}
}

View File

@ -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());
}