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; package org.apache.hadoop.hbase.client;
import org.apache.hadoop.classification.InterfaceAudience; 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 * {@link HTable#batch} to associate the action with it's region and maintain
* the index from the original request. * 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. // TODO: This class should not be visible outside of the client package.
private Row action; private Row action;
private int originalIndex; private int originalIndex;
private long nonce = HConstants.NO_NONCE;
public Action(Row action, int originalIndex) { public Action(Row action, int originalIndex) {
super(); super();
@ -37,6 +39,13 @@ public class Action<R> implements Comparable<R> {
this.originalIndex = originalIndex; this.originalIndex = originalIndex;
} }
public void setNonce(long nonce) {
this.nonce = nonce;
}
public boolean hasNonce() {
return nonce != HConstants.NO_NONCE;
}
public Row getAction() { public Row getAction() {
return action; return action;
@ -64,4 +73,8 @@ public class Action<R> implements Comparable<R> {
Action<?> other = (Action<?>) obj; Action<?> other = (Action<?>) obj;
return compareTo(other) == 0; return compareTo(other) == 0;
} }
public long getNonce() {
return nonce;
}
} }

View File

@ -275,6 +275,7 @@ class AsyncProcess<CResult> {
long currentTaskCnt = tasksDone.get(); long currentTaskCnt = tasksDone.get();
boolean alreadyLooped = false; boolean alreadyLooped = false;
NonceGenerator ng = this.hConnection.getNonceGenerator();
do { do {
if (alreadyLooped){ if (alreadyLooped){
// if, for whatever reason, we looped, we want to be sure that something has changed. // if, for whatever reason, we looped, we want to be sure that something has changed.
@ -302,12 +303,12 @@ class AsyncProcess<CResult> {
it.remove(); it.remove();
} else if (canTakeOperation(loc, regionIncluded, serverIncluded)) { } else if (canTakeOperation(loc, regionIncluded, serverIncluded)) {
Action<Row> action = new Action<Row>(r, ++posInList); Action<Row> action = new Action<Row>(r, ++posInList);
setNonce(ng, r, action);
retainedActions.add(action); retainedActions.add(action);
addAction(loc, action, actionsByServer); addAction(loc, action, actionsByServer, ng);
it.remove(); it.remove();
} }
} }
} while (retainedActions.isEmpty() && atLeastOne && !hasError()); } while (retainedActions.isEmpty() && atLeastOne && !hasError());
HConnectionManager.ServerErrorTracker errorsByServer = createServerErrorTracker(); HConnectionManager.ServerErrorTracker errorsByServer = createServerErrorTracker();
@ -320,15 +321,21 @@ class AsyncProcess<CResult> {
* @param loc - the destination. Must not be null. * @param loc - the destination. Must not be null.
* @param action - the action to add to the multiaction * @param action - the action to add to the multiaction
* @param actionsByServer the multiaction per server * @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, private void addAction(HRegionLocation loc, Action<Row> action, Map<HRegionLocation,
MultiAction<Row>> actionsByServer) { MultiAction<Row>> actionsByServer, NonceGenerator ng) {
final byte[] regionName = loc.getRegionInfo().getRegionName(); final byte[] regionName = loc.getRegionInfo().getRegionName();
MultiAction<Row> multiAction = actionsByServer.get(loc); MultiAction<Row> multiAction = actionsByServer.get(loc);
if (multiAction == null) { if (multiAction == null) {
multiAction = new MultiAction<Row>(); multiAction = new MultiAction<Row>();
actionsByServer.put(loc, multiAction); 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); 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. // The position will be used by the processBatch to match the object array returned.
int posInList = -1; int posInList = -1;
NonceGenerator ng = this.hConnection.getNonceGenerator();
for (Row r : rows) { for (Row r : rows) {
posInList++; posInList++;
Action<Row> action = new Action<Row>(r, posInList); Action<Row> action = new Action<Row>(r, posInList);
setNonce(ng, r, action);
actions.add(action); actions.add(action);
} }
HConnectionManager.ServerErrorTracker errorsByServer = createServerErrorTracker(); HConnectionManager.ServerErrorTracker errorsByServer = createServerErrorTracker();
submit(actions, actions, 1, errorsByServer); 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 * 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 = final Map<HRegionLocation, MultiAction<Row>> actionsByServer =
new HashMap<HRegionLocation, MultiAction<Row>>(); new HashMap<HRegionLocation, MultiAction<Row>>();
NonceGenerator ng = this.hConnection.getNonceGenerator();
for (Action<Row> action : currentActions) { for (Action<Row> action : currentActions) {
HRegionLocation loc = findDestLocation(action.getAction(), action.getOriginalIndex()); HRegionLocation loc = findDestLocation(action.getAction(), action.getOriginalIndex());
if (loc != null) { 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. * @return true if the server is known as dead, false otherwise.
*/ */
boolean isDeadServer(ServerName serverName); 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.hadoop.ipc.RemoteException;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.BlockingRpcChannel; import com.google.protobuf.BlockingRpcChannel;
import com.google.protobuf.RpcController; import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException; import com.google.protobuf.ServiceException;
@ -197,6 +198,7 @@ public class HConnectionManager {
static final Log LOG = LogFactory.getLog(HConnectionManager.class); static final Log LOG = LogFactory.getLog(HConnectionManager.class);
public static final String RETRIES_BY_SERVER_KEY = "hbase.client.retries.by.server"; 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 // An LRU Map of HConnectionKey -> HConnection (TableServer). All
// access must be synchronized. This map is not private because tests // 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; 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 { static {
// We set instances to one more than the value specified for {@link // We set instances to one more than the value specified for {@link
// HConstants#ZOOKEEPER_MAX_CLIENT_CNXNS}. By default, the zk default max // HConstants#ZOOKEEPER_MAX_CLIENT_CNXNS}. By default, the zk default max
@ -229,6 +239,20 @@ public class HConnectionManager {
super(); 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. * 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 * 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 long pause;
private final int numTries; private final int numTries;
final int rpcTimeout; final int rpcTimeout;
private NonceGenerator nonceGenerator = null;
private final int prefetchRegionLimit; private final int prefetchRegionLimit;
private volatile boolean closed; 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. * For tests.
@ -675,6 +711,17 @@ public class HConnectionManager {
this.rpcTimeout = conf.getInt( this.rpcTimeout = conf.getInt(
HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.HBASE_RPC_TIMEOUT_KEY,
HConstants.DEFAULT_HBASE_RPC_TIMEOUT); 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( this.prefetchRegionLimit = conf.getInt(
HConstants.HBASE_CLIENT_PREFETCH_LIMIT, HConstants.HBASE_CLIENT_PREFETCH_LIMIT,
HConstants.DEFAULT_HBASE_CLIENT_PREFETCH_LIMIT); HConstants.DEFAULT_HBASE_CLIENT_PREFETCH_LIMIT);
@ -2617,6 +2664,11 @@ public class HConnectionManager {
return getHTableDescriptorsByTableName(tableNames); return getHTableDescriptorsByTableName(tableNames);
} }
@Override
public NonceGenerator getNonceGenerator() {
return this.nonceGenerator;
}
/** /**
* Connects to the master to get the table descriptor. * Connects to the master to get the table descriptor.
* @param tableName table name * @param tableName table name

View File

@ -986,12 +986,15 @@ public class HTable implements HTableInterface {
throw new IOException( throw new IOException(
"Invalid arguments to append, no columns specified"); "Invalid arguments to append, no columns specified");
} }
NonceGenerator ng = this.connection.getNonceGenerator();
final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
RegionServerCallable<Result> callable = RegionServerCallable<Result> callable =
new RegionServerCallable<Result>(this.connection, getName(), append.getRow()) { new RegionServerCallable<Result>(this.connection, getName(), append.getRow()) {
public Result call() throws IOException { public Result call() throws IOException {
try { try {
MutateRequest request = RequestConverter.buildMutateRequest( MutateRequest request = RequestConverter.buildMutateRequest(
getLocation().getRegionInfo().getRegionName(), append); getLocation().getRegionInfo().getRegionName(), append, nonceGroup, nonce);
PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController(); PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
rpcController.setPriority(getTableName()); rpcController.setPriority(getTableName());
MutateResponse response = getStub().mutate(rpcController, request); MutateResponse response = getStub().mutate(rpcController, request);
@ -1014,19 +1017,21 @@ public class HTable implements HTableInterface {
throw new IOException( throw new IOException(
"Invalid arguments to increment, no columns specified"); "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, RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
getName(), increment.getRow()) { getName(), increment.getRow()) {
public Result call() throws IOException { public Result call() throws IOException {
try { try {
MutateRequest request = RequestConverter.buildMutateRequest( MutateRequest request = RequestConverter.buildMutateRequest(
getLocation().getRegionInfo().getRegionName(), increment); getLocation().getRegionInfo().getRegionName(), increment, nonceGroup, nonce);
PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController(); PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
rpcController.setPriority(getTableName()); rpcController.setPriority(getTableName());
MutateResponse response = getStub().mutate(rpcController, request); MutateResponse response = getStub().mutate(rpcController, request);
return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner()); return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
} catch (ServiceException se) { } catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se); throw ProtobufUtil.getRemoteException(se);
} }
} }
}; };
return rpcCallerFactory.<Result> newCaller().callWithRetries(callable, this.operationTimeout); return rpcCallerFactory.<Result> newCaller().callWithRetries(callable, this.operationTimeout);
@ -1074,13 +1079,15 @@ public class HTable implements HTableInterface {
"Invalid arguments to incrementColumnValue", npe); "Invalid arguments to incrementColumnValue", npe);
} }
NonceGenerator ng = this.connection.getNonceGenerator();
final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
RegionServerCallable<Long> callable = RegionServerCallable<Long> callable =
new RegionServerCallable<Long>(connection, getName(), row) { new RegionServerCallable<Long>(connection, getName(), row) {
public Long call() throws IOException { public Long call() throws IOException {
try { try {
MutateRequest request = RequestConverter.buildMutateRequest( MutateRequest request = RequestConverter.buildIncrementRequest(
getLocation().getRegionInfo().getRegionName(), row, family, getLocation().getRegionInfo().getRegionName(), row, family,
qualifier, amount, durability); qualifier, amount, durability, nonceGroup, nonce);
PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController(); PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
rpcController.setPriority(getTableName()); rpcController.setPriority(getTableName());
MutateResponse response = getStub().mutate(rpcController, request); MutateResponse response = getStub().mutate(rpcController, request);

View File

@ -25,6 +25,7 @@ import java.util.Set;
import java.util.TreeMap; import java.util.TreeMap;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
/** /**
@ -39,6 +40,8 @@ public final class MultiAction<R> {
public Map<byte[], List<Action<R>>> actions = public Map<byte[], List<Action<R>>> actions =
new TreeMap<byte[], List<Action<R>>>(Bytes.BYTES_COMPARATOR); new TreeMap<byte[], List<Action<R>>>(Bytes.BYTES_COMPARATOR);
private long nonceGroup = HConstants.NO_NONCE;
public MultiAction() { public MultiAction() {
super(); super();
} }
@ -73,6 +76,10 @@ public final class MultiAction<R> {
rsActions.add(a); rsActions.add(a);
} }
public void setNonceGroup(long nonceGroup) {
this.nonceGroup = nonceGroup;
}
public Set<byte[]> getRegions() { public Set<byte[]> getRegions() {
return actions.keySet(); return actions.keySet();
} }
@ -87,4 +94,12 @@ public final class MultiAction<R> {
} }
return res; 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.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellScannable; import org.apache.hadoop.hbase.CellScannable;
import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation; import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
@ -71,6 +73,10 @@ class MultiServerCallable<R> extends RegionServerCallable<MultiResponse> {
MutationProto.Builder mutationBuilder = MutationProto.newBuilder(); MutationProto.Builder mutationBuilder = MutationProto.newBuilder();
List<CellScannable> cells = null; List<CellScannable> cells = null;
// The multi object is a list of Actions by region. Iterate by region. // 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()) { for (Map.Entry<byte[], List<Action<R>>> e: this.multiAction.actions.entrySet()) {
final byte [] regionName = e.getKey(); final byte [] regionName = e.getKey();
final List<Action<R>> actions = e.getValue(); final List<Action<R>> actions = e.getValue();
@ -92,6 +98,7 @@ class MultiServerCallable<R> extends RegionServerCallable<MultiResponse> {
} }
multiRequestBuilder.addRegionAction(regionActionBuilder.build()); multiRequestBuilder.addRegionAction(regionActionBuilder.build());
} }
// Controller optionally carries cell data over the proxy/service boundary and also // Controller optionally carries cell data over the proxy/service boundary and also
// optionally ferries cell response data back out again. // optionally ferries cell response data back out again.
PayloadCarryingRpcController controller = new PayloadCarryingRpcController(cells); 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 * @param increment
* @return the converted mutate * @return the converted mutate
*/ */
public static MutationProto toMutation(final Increment increment, public static MutationProto toMutation(
final MutationProto.Builder builder) { final Increment increment, final MutationProto.Builder builder, long nonce) {
builder.setRow(ZeroCopyLiteralByteString.wrap(increment.getRow())); builder.setRow(ZeroCopyLiteralByteString.wrap(increment.getRow()));
builder.setMutateType(MutationType.INCREMENT); builder.setMutateType(MutationType.INCREMENT);
builder.setDurability(toDurability(increment.getDurability())); builder.setDurability(toDurability(increment.getDurability()));
if (nonce != HConstants.NO_NONCE) {
builder.setNonce(nonce);
}
TimeRange timeRange = increment.getTimeRange(); TimeRange timeRange = increment.getTimeRange();
if (!timeRange.isAllTime()) { if (!timeRange.isAllTime()) {
HBaseProtos.TimeRange.Builder timeRangeBuilder = HBaseProtos.TimeRange.Builder timeRangeBuilder =
@ -1035,6 +1038,11 @@ public final class ProtobufUtil {
return builder.build(); 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 * Create a protocol buffer Mutate based on a client Mutation
* *
@ -1043,15 +1051,23 @@ public final class ProtobufUtil {
* @return a protobuf'd Mutation * @return a protobuf'd Mutation
* @throws IOException * @throws IOException
*/ */
public static MutationProto toMutation(final MutationType type, final Mutation mutation) public static MutationProto toMutation(final MutationType type, final Mutation mutation,
throws IOException { final long nonce) throws IOException {
return toMutation(type, mutation, MutationProto.newBuilder()); return toMutation(type, mutation, MutationProto.newBuilder(), nonce);
} }
public static MutationProto toMutation(final MutationType type, final Mutation mutation, 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 { throws IOException {
builder = getMutationBuilderAndSetCommonFields(type, mutation, builder); builder = getMutationBuilderAndSetCommonFields(type, mutation, builder);
if (nonce != HConstants.NO_NONCE) {
builder.setNonce(nonce);
}
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder(); ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder(); QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
for (Map.Entry<byte[],List<Cell>> family: mutation.getFamilyCellMap().entrySet()) { for (Map.Entry<byte[],List<Cell>> family: mutation.getFamilyCellMap().entrySet()) {
@ -1078,6 +1094,11 @@ public final class ProtobufUtil {
return builder.build(); 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. * 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. * Understanding is that the Cell will be transported other than via protobuf.
@ -1087,10 +1108,12 @@ public final class ProtobufUtil {
* @throws IOException * @throws IOException
*/ */
public static MutationProto toMutationNoData(final MutationType type, final Mutation mutation, public static MutationProto toMutationNoData(final MutationType type, final Mutation mutation,
final MutationProto.Builder builder) final MutationProto.Builder builder, long nonce) throws IOException {
throws IOException {
getMutationBuilderAndSetCommonFields(type, mutation, builder); getMutationBuilderAndSetCommonFields(type, mutation, builder);
builder.setAssociatedCellCount(mutation.size()); builder.setAssociatedCellCount(mutation.size());
if (nonce != HConstants.NO_NONCE) {
builder.setNonce(nonce);
}
return builder.build(); return builder.build();
} }

View File

@ -22,6 +22,8 @@ import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CellScannable; 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.DoNotRetryIOException;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
@ -172,9 +174,9 @@ public final class RequestConverter {
* @param durability * @param durability
* @return a mutate request * @return a mutate request
*/ */
public static MutateRequest buildMutateRequest( public static MutateRequest buildIncrementRequest(
final byte[] regionName, final byte[] row, final byte[] family, final byte[] regionName, final byte[] row, final byte[] family, final byte[] qualifier,
final byte [] qualifier, final long amount, final Durability durability) { final long amount, final Durability durability, long nonceGroup, long nonce) {
MutateRequest.Builder builder = MutateRequest.newBuilder(); MutateRequest.Builder builder = MutateRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier( RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName); RegionSpecifierType.REGION_NAME, regionName);
@ -191,7 +193,13 @@ public final class RequestConverter {
valueBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier)); valueBuilder.setQualifier(ZeroCopyLiteralByteString.wrap(qualifier));
columnBuilder.addQualifierValue(valueBuilder.build()); columnBuilder.addQualifierValue(valueBuilder.build());
mutateBuilder.addColumnValue(columnBuilder.build()); mutateBuilder.addColumnValue(columnBuilder.build());
if (nonce != HConstants.NO_NONCE) {
mutateBuilder.setNonce(nonce);
}
builder.setMutation(mutateBuilder.build()); builder.setMutation(mutateBuilder.build());
if (nonceGroup != HConstants.NO_NONCE) {
builder.setNonceGroup(nonceGroup);
}
return builder.build(); return builder.build();
} }
@ -278,14 +286,17 @@ public final class RequestConverter {
* @return a mutate request * @return a mutate request
* @throws IOException * @throws IOException
*/ */
public static MutateRequest buildMutateRequest( public static MutateRequest buildMutateRequest(final byte[] regionName,
final byte[] regionName, final Append append) throws IOException { final Append append, long nonceGroup, long nonce) throws IOException {
MutateRequest.Builder builder = MutateRequest.newBuilder(); MutateRequest.Builder builder = MutateRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier( RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName); RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region); builder.setRegion(region);
if (nonce != HConstants.NO_NONCE && nonceGroup != HConstants.NO_NONCE) {
builder.setNonceGroup(nonceGroup);
}
builder.setMutation(ProtobufUtil.toMutation(MutationType.APPEND, append, builder.setMutation(ProtobufUtil.toMutation(MutationType.APPEND, append,
MutationProto.newBuilder())); MutationProto.newBuilder(), nonce));
return builder.build(); return builder.build();
} }
@ -296,13 +307,16 @@ public final class RequestConverter {
* @param increment * @param increment
* @return a mutate request * @return a mutate request
*/ */
public static MutateRequest buildMutateRequest( public static MutateRequest buildMutateRequest(final byte[] regionName,
final byte[] regionName, final Increment increment) { final Increment increment, final long nonceGroup, final long nonce) {
MutateRequest.Builder builder = MutateRequest.newBuilder(); MutateRequest.Builder builder = MutateRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier( RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName); RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region); 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(); return builder.build();
} }
@ -499,8 +513,7 @@ public final class RequestConverter {
public static <R> RegionAction.Builder buildRegionAction(final byte[] regionName, public static <R> RegionAction.Builder buildRegionAction(final byte[] regionName,
final List<Action<R>> actions, final RegionAction.Builder regionActionBuilder, final List<Action<R>> actions, final RegionAction.Builder regionActionBuilder,
final ClientProtos.Action.Builder actionBuilder, final ClientProtos.Action.Builder actionBuilder,
final MutationProto.Builder mutationBuilder) final MutationProto.Builder mutationBuilder) throws IOException {
throws IOException {
for (Action<R> action: actions) { for (Action<R> action: actions) {
Row row = action.getAction(); Row row = action.getAction();
actionBuilder.clear(); actionBuilder.clear();
@ -516,11 +529,11 @@ public final class RequestConverter {
regionActionBuilder.addAction(actionBuilder. regionActionBuilder.addAction(actionBuilder.
setMutation(ProtobufUtil.toMutation(MutationType.DELETE, (Delete)row, mutationBuilder))); setMutation(ProtobufUtil.toMutation(MutationType.DELETE, (Delete)row, mutationBuilder)));
} else if (row instanceof Append) { } else if (row instanceof Append) {
regionActionBuilder.addAction(actionBuilder. regionActionBuilder.addAction(actionBuilder.setMutation(ProtobufUtil.toMutation(
setMutation(ProtobufUtil.toMutation(MutationType.APPEND, (Append)row, mutationBuilder))); MutationType.APPEND, (Append)row, mutationBuilder, action.getNonce())));
} else if (row instanceof Increment) { } else if (row instanceof Increment) {
regionActionBuilder.addAction(actionBuilder. regionActionBuilder.addAction(actionBuilder.setMutation(
setMutation(ProtobufUtil.toMutation((Increment)row, mutationBuilder))); ProtobufUtil.toMutation((Increment)row, mutationBuilder, action.getNonce())));
} else if (row instanceof RowMutations) { } else if (row instanceof RowMutations) {
throw new UnsupportedOperationException("No RowMutations in multi calls; use mutateRow"); throw new UnsupportedOperationException("No RowMutations in multi calls; use mutateRow");
} else { } else {
@ -550,10 +563,9 @@ public final class RequestConverter {
final List<Action<R>> actions, final List<CellScannable> cells, final List<Action<R>> actions, final List<CellScannable> cells,
final RegionAction.Builder regionActionBuilder, final RegionAction.Builder regionActionBuilder,
final ClientProtos.Action.Builder actionBuilder, final ClientProtos.Action.Builder actionBuilder,
final MutationProto.Builder mutationBuilder) final MutationProto.Builder mutationBuilder) throws IOException {
throws IOException { RegionAction.Builder builder = getRegionActionBuilderWithRegion(
RegionAction.Builder builder = RegionAction.newBuilder(), regionName);
getRegionActionBuilderWithRegion(RegionAction.newBuilder(), regionName);
for (Action<R> action: actions) { for (Action<R> action: actions) {
Row row = action.getAction(); Row row = action.getAction();
actionBuilder.clear(); actionBuilder.clear();
@ -586,13 +598,13 @@ public final class RequestConverter {
} else if (row instanceof Append) { } else if (row instanceof Append) {
Append a = (Append)row; Append a = (Append)row;
cells.add(a); cells.add(a);
builder.addAction(actionBuilder. builder.addAction(actionBuilder.setMutation(ProtobufUtil.toMutationNoData(
setMutation(ProtobufUtil.toMutationNoData(MutationType.APPEND, a, mutationBuilder))); MutationType.APPEND, a, mutationBuilder, action.getNonce())));
} else if (row instanceof Increment) { } else if (row instanceof Increment) {
Increment i = (Increment)row; Increment i = (Increment)row;
cells.add(i); cells.add(i);
builder.addAction(actionBuilder. builder.addAction(actionBuilder.setMutation(ProtobufUtil.toMutationNoData(
setMutation(ProtobufUtil.toMutationNoData(MutationType.INCREMENT, i, mutationBuilder))); MutationType.INCREMENT, i, mutationBuilder, action.getNonce())));
} else if (row instanceof RowMutations) { } else if (row instanceof RowMutations) {
continue; // ignore RowMutations continue; // ignore RowMutations
} else { } else {

View File

@ -743,6 +743,9 @@ public final class HConstants {
public static final String HBASE_REGION_SPLIT_POLICY_KEY = public static final String HBASE_REGION_SPLIT_POLICY_KEY =
"hbase.regionserver.region.split.policy"; "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 * 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 String STATUS_MULTICAST_PORT = "hbase.status.multicast.port";
public static final int DEFAULT_STATUS_MULTICAST_PORT = 60100; public static final int DEFAULT_STATUS_MULTICAST_PORT = 60100;
public static final long NO_NONCE = 0;
private HConstants() { private HConstants() {
// Can't be instantiated with this ctor. // Can't be instantiated with this ctor.
} }

View File

@ -35,6 +35,11 @@ public class Triple<A, B, C> {
this.third = third; 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() { public int hashCode() {
int hashFirst = (first != null ? first.hashCode() : 0); int hashFirst = (first != null ? first.hashCode() : 0);
int hashSecond = (second != null ? second.hashCode() : 0); int hashSecond = (second != null ? second.hashCode() : 0);

View File

@ -6487,6 +6487,16 @@ public final class ClientProtos {
* </pre> * </pre>
*/ */
int getAssociatedCellCount(); 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} * Protobuf type {@code MutationProto}
@ -6614,6 +6624,11 @@ public final class ClientProtos {
associatedCellCount_ = input.readInt32(); associatedCellCount_ = input.readInt32();
break; break;
} }
case 72: {
bitField0_ |= 0x00000040;
nonce_ = input.readUInt64();
break;
}
} }
} }
} catch (com.google.protobuf.InvalidProtocolBufferException e) { } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@ -8801,6 +8816,22 @@ public final class ClientProtos {
return associatedCellCount_; 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() { private void initFields() {
row_ = com.google.protobuf.ByteString.EMPTY; row_ = com.google.protobuf.ByteString.EMPTY;
mutateType_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType.APPEND; 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; durability_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Durability.USE_DEFAULT;
timeRange_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.getDefaultInstance(); timeRange_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TimeRange.getDefaultInstance();
associatedCellCount_ = 0; associatedCellCount_ = 0;
nonce_ = 0L;
} }
private byte memoizedIsInitialized = -1; private byte memoizedIsInitialized = -1;
public final boolean isInitialized() { public final boolean isInitialized() {
@ -8859,6 +8891,9 @@ public final class ClientProtos {
if (((bitField0_ & 0x00000020) == 0x00000020)) { if (((bitField0_ & 0x00000020) == 0x00000020)) {
output.writeInt32(8, associatedCellCount_); output.writeInt32(8, associatedCellCount_);
} }
if (((bitField0_ & 0x00000040) == 0x00000040)) {
output.writeUInt64(9, nonce_);
}
getUnknownFields().writeTo(output); getUnknownFields().writeTo(output);
} }
@ -8900,6 +8935,10 @@ public final class ClientProtos {
size += com.google.protobuf.CodedOutputStream size += com.google.protobuf.CodedOutputStream
.computeInt32Size(8, associatedCellCount_); .computeInt32Size(8, associatedCellCount_);
} }
if (((bitField0_ & 0x00000040) == 0x00000040)) {
size += com.google.protobuf.CodedOutputStream
.computeUInt64Size(9, nonce_);
}
size += getUnknownFields().getSerializedSize(); size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size; memoizedSerializedSize = size;
return size; return size;
@ -8957,6 +8996,11 @@ public final class ClientProtos {
result = result && (getAssociatedCellCount() result = result && (getAssociatedCellCount()
== other.getAssociatedCellCount()); == other.getAssociatedCellCount());
} }
result = result && (hasNonce() == other.hasNonce());
if (hasNonce()) {
result = result && (getNonce()
== other.getNonce());
}
result = result && result = result &&
getUnknownFields().equals(other.getUnknownFields()); getUnknownFields().equals(other.getUnknownFields());
return result; return result;
@ -9002,6 +9046,10 @@ public final class ClientProtos {
hash = (37 * hash) + ASSOCIATED_CELL_COUNT_FIELD_NUMBER; hash = (37 * hash) + ASSOCIATED_CELL_COUNT_FIELD_NUMBER;
hash = (53 * hash) + getAssociatedCellCount(); hash = (53 * hash) + getAssociatedCellCount();
} }
if (hasNonce()) {
hash = (37 * hash) + NONCE_FIELD_NUMBER;
hash = (53 * hash) + hashLong(getNonce());
}
hash = (29 * hash) + getUnknownFields().hashCode(); hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash; memoizedHashCode = hash;
return hash; return hash;
@ -9151,6 +9199,8 @@ public final class ClientProtos {
bitField0_ = (bitField0_ & ~0x00000040); bitField0_ = (bitField0_ & ~0x00000040);
associatedCellCount_ = 0; associatedCellCount_ = 0;
bitField0_ = (bitField0_ & ~0x00000080); bitField0_ = (bitField0_ & ~0x00000080);
nonce_ = 0L;
bitField0_ = (bitField0_ & ~0x00000100);
return this; return this;
} }
@ -9225,6 +9275,10 @@ public final class ClientProtos {
to_bitField0_ |= 0x00000020; to_bitField0_ |= 0x00000020;
} }
result.associatedCellCount_ = associatedCellCount_; result.associatedCellCount_ = associatedCellCount_;
if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
to_bitField0_ |= 0x00000040;
}
result.nonce_ = nonce_;
result.bitField0_ = to_bitField0_; result.bitField0_ = to_bitField0_;
onBuilt(); onBuilt();
return result; return result;
@ -9311,6 +9365,9 @@ public final class ClientProtos {
if (other.hasAssociatedCellCount()) { if (other.hasAssociatedCellCount()) {
setAssociatedCellCount(other.getAssociatedCellCount()); setAssociatedCellCount(other.getAssociatedCellCount());
} }
if (other.hasNonce()) {
setNonce(other.getNonce());
}
this.mergeUnknownFields(other.getUnknownFields()); this.mergeUnknownFields(other.getUnknownFields());
return this; return this;
} }
@ -10202,6 +10259,39 @@ public final class ClientProtos {
return this; 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) // @@protoc_insertion_point(builder_scope:MutationProto)
} }
@ -10257,6 +10347,16 @@ public final class ClientProtos {
* <code>optional .Condition condition = 3;</code> * <code>optional .Condition condition = 3;</code>
*/ */
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ConditionOrBuilder getConditionOrBuilder(); 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} * Protobuf type {@code MutateRequest}
@ -10359,6 +10459,11 @@ public final class ClientProtos {
bitField0_ |= 0x00000004; bitField0_ |= 0x00000004;
break; break;
} }
case 32: {
bitField0_ |= 0x00000008;
nonceGroup_ = input.readUInt64();
break;
}
} }
} }
} catch (com.google.protobuf.InvalidProtocolBufferException e) { } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@ -10465,10 +10570,27 @@ public final class ClientProtos {
return condition_; 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() { private void initFields() {
region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance(); region_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.getDefaultInstance();
mutation_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance(); mutation_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.getDefaultInstance();
condition_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Condition.getDefaultInstance(); condition_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Condition.getDefaultInstance();
nonceGroup_ = 0L;
} }
private byte memoizedIsInitialized = -1; private byte memoizedIsInitialized = -1;
public final boolean isInitialized() { public final boolean isInitialized() {
@ -10513,6 +10635,9 @@ public final class ClientProtos {
if (((bitField0_ & 0x00000004) == 0x00000004)) { if (((bitField0_ & 0x00000004) == 0x00000004)) {
output.writeMessage(3, condition_); output.writeMessage(3, condition_);
} }
if (((bitField0_ & 0x00000008) == 0x00000008)) {
output.writeUInt64(4, nonceGroup_);
}
getUnknownFields().writeTo(output); getUnknownFields().writeTo(output);
} }
@ -10534,6 +10659,10 @@ public final class ClientProtos {
size += com.google.protobuf.CodedOutputStream size += com.google.protobuf.CodedOutputStream
.computeMessageSize(3, condition_); .computeMessageSize(3, condition_);
} }
if (((bitField0_ & 0x00000008) == 0x00000008)) {
size += com.google.protobuf.CodedOutputStream
.computeUInt64Size(4, nonceGroup_);
}
size += getUnknownFields().getSerializedSize(); size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size; memoizedSerializedSize = size;
return size; return size;
@ -10572,6 +10701,11 @@ public final class ClientProtos {
result = result && getCondition() result = result && getCondition()
.equals(other.getCondition()); .equals(other.getCondition());
} }
result = result && (hasNonceGroup() == other.hasNonceGroup());
if (hasNonceGroup()) {
result = result && (getNonceGroup()
== other.getNonceGroup());
}
result = result && result = result &&
getUnknownFields().equals(other.getUnknownFields()); getUnknownFields().equals(other.getUnknownFields());
return result; return result;
@ -10597,6 +10731,10 @@ public final class ClientProtos {
hash = (37 * hash) + CONDITION_FIELD_NUMBER; hash = (37 * hash) + CONDITION_FIELD_NUMBER;
hash = (53 * hash) + getCondition().hashCode(); hash = (53 * hash) + getCondition().hashCode();
} }
if (hasNonceGroup()) {
hash = (37 * hash) + NONCE_GROUP_FIELD_NUMBER;
hash = (53 * hash) + hashLong(getNonceGroup());
}
hash = (29 * hash) + getUnknownFields().hashCode(); hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash; memoizedHashCode = hash;
return hash; return hash;
@ -10738,6 +10876,8 @@ public final class ClientProtos {
conditionBuilder_.clear(); conditionBuilder_.clear();
} }
bitField0_ = (bitField0_ & ~0x00000004); bitField0_ = (bitField0_ & ~0x00000004);
nonceGroup_ = 0L;
bitField0_ = (bitField0_ & ~0x00000008);
return this; return this;
} }
@ -10790,6 +10930,10 @@ public final class ClientProtos {
} else { } else {
result.condition_ = conditionBuilder_.build(); result.condition_ = conditionBuilder_.build();
} }
if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
to_bitField0_ |= 0x00000008;
}
result.nonceGroup_ = nonceGroup_;
result.bitField0_ = to_bitField0_; result.bitField0_ = to_bitField0_;
onBuilt(); onBuilt();
return result; return result;
@ -10815,6 +10959,9 @@ public final class ClientProtos {
if (other.hasCondition()) { if (other.hasCondition()) {
mergeCondition(other.getCondition()); mergeCondition(other.getCondition());
} }
if (other.hasNonceGroup()) {
setNonceGroup(other.getNonceGroup());
}
this.mergeUnknownFields(other.getUnknownFields()); this.mergeUnknownFields(other.getUnknownFields());
return this; return this;
} }
@ -11215,6 +11362,39 @@ public final class ClientProtos {
return conditionBuilder_; 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) // @@protoc_insertion_point(builder_scope:MutateRequest)
} }
@ -25450,6 +25630,16 @@ public final class ClientProtos {
*/ */
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionOrBuilder getRegionActionOrBuilder( org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionActionOrBuilder getRegionActionOrBuilder(
int index); 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} * 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)); regionAction_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction.PARSER, extensionRegistry));
break; break;
} }
case 16: {
bitField0_ |= 0x00000001;
nonceGroup_ = input.readUInt64();
break;
}
} }
} }
} catch (com.google.protobuf.InvalidProtocolBufferException e) { } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@ -25561,6 +25756,7 @@ public final class ClientProtos {
return PARSER; return PARSER;
} }
private int bitField0_;
// repeated .RegionAction regionAction = 1; // repeated .RegionAction regionAction = 1;
public static final int REGIONACTION_FIELD_NUMBER = 1; public static final int REGIONACTION_FIELD_NUMBER = 1;
private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction> regionAction_; 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); 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() { private void initFields() {
regionAction_ = java.util.Collections.emptyList(); regionAction_ = java.util.Collections.emptyList();
nonceGroup_ = 0L;
} }
private byte memoizedIsInitialized = -1; private byte memoizedIsInitialized = -1;
public final boolean isInitialized() { public final boolean isInitialized() {
@ -25621,6 +25834,9 @@ public final class ClientProtos {
for (int i = 0; i < regionAction_.size(); i++) { for (int i = 0; i < regionAction_.size(); i++) {
output.writeMessage(1, regionAction_.get(i)); output.writeMessage(1, regionAction_.get(i));
} }
if (((bitField0_ & 0x00000001) == 0x00000001)) {
output.writeUInt64(2, nonceGroup_);
}
getUnknownFields().writeTo(output); getUnknownFields().writeTo(output);
} }
@ -25634,6 +25850,10 @@ public final class ClientProtos {
size += com.google.protobuf.CodedOutputStream size += com.google.protobuf.CodedOutputStream
.computeMessageSize(1, regionAction_.get(i)); .computeMessageSize(1, regionAction_.get(i));
} }
if (((bitField0_ & 0x00000001) == 0x00000001)) {
size += com.google.protobuf.CodedOutputStream
.computeUInt64Size(2, nonceGroup_);
}
size += getUnknownFields().getSerializedSize(); size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size; memoizedSerializedSize = size;
return size; return size;
@ -25659,6 +25879,11 @@ public final class ClientProtos {
boolean result = true; boolean result = true;
result = result && getRegionActionList() result = result && getRegionActionList()
.equals(other.getRegionActionList()); .equals(other.getRegionActionList());
result = result && (hasNonceGroup() == other.hasNonceGroup());
if (hasNonceGroup()) {
result = result && (getNonceGroup()
== other.getNonceGroup());
}
result = result && result = result &&
getUnknownFields().equals(other.getUnknownFields()); getUnknownFields().equals(other.getUnknownFields());
return result; return result;
@ -25676,6 +25901,10 @@ public final class ClientProtos {
hash = (37 * hash) + REGIONACTION_FIELD_NUMBER; hash = (37 * hash) + REGIONACTION_FIELD_NUMBER;
hash = (53 * hash) + getRegionActionList().hashCode(); hash = (53 * hash) + getRegionActionList().hashCode();
} }
if (hasNonceGroup()) {
hash = (37 * hash) + NONCEGROUP_FIELD_NUMBER;
hash = (53 * hash) + hashLong(getNonceGroup());
}
hash = (29 * hash) + getUnknownFields().hashCode(); hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash; memoizedHashCode = hash;
return hash; return hash;
@ -25801,6 +26030,8 @@ public final class ClientProtos {
} else { } else {
regionActionBuilder_.clear(); regionActionBuilder_.clear();
} }
nonceGroup_ = 0L;
bitField0_ = (bitField0_ & ~0x00000002);
return this; return this;
} }
@ -25828,6 +26059,7 @@ public final class ClientProtos {
public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest buildPartial() { 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); org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest result = new org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest(this);
int from_bitField0_ = bitField0_; int from_bitField0_ = bitField0_;
int to_bitField0_ = 0;
if (regionActionBuilder_ == null) { if (regionActionBuilder_ == null) {
if (((bitField0_ & 0x00000001) == 0x00000001)) { if (((bitField0_ & 0x00000001) == 0x00000001)) {
regionAction_ = java.util.Collections.unmodifiableList(regionAction_); regionAction_ = java.util.Collections.unmodifiableList(regionAction_);
@ -25837,6 +26069,11 @@ public final class ClientProtos {
} else { } else {
result.regionAction_ = regionActionBuilder_.build(); result.regionAction_ = regionActionBuilder_.build();
} }
if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
to_bitField0_ |= 0x00000001;
}
result.nonceGroup_ = nonceGroup_;
result.bitField0_ = to_bitField0_;
onBuilt(); onBuilt();
return result; return result;
} }
@ -25878,6 +26115,9 @@ public final class ClientProtos {
} }
} }
} }
if (other.hasNonceGroup()) {
setNonceGroup(other.getNonceGroup());
}
this.mergeUnknownFields(other.getUnknownFields()); this.mergeUnknownFields(other.getUnknownFields());
return this; return this;
} }
@ -26151,6 +26391,39 @@ public final class ClientProtos {
return regionActionBuilder_; 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) // @@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" + "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" + " \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" + "\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", "\001(\014\0220\n\013mutate_type\030\002 \001(\0162\033.MutationProto",
".MutationType\0220\n\014column_value\030\003 \003(\0132\032.Mu" + ".MutationType\0220\n\014column_value\030\003 \003(\0132\032.Mu" +
"tationProto.ColumnValue\022\021\n\ttimestamp\030\004 \001" + "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" + "\ndurability\030\006 \001(\0162\031.MutationProto.Durabi" +
"lity:\013USE_DEFAULT\022\036\n\ntime_range\030\007 \001(\0132\n." + "lity:\013USE_DEFAULT\022\036\n\ntime_range\030\007 \001(\0132\n." +
"TimeRange\022\035\n\025associated_cell_count\030\010 \001(\005" + "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" + "\022\r\n\005nonce\030\t \001(\004\032\347\001\n\013ColumnValue\022\016\n\006famil" +
"ifier_value\030\002 \003(\0132).MutationProto.Column" + "y\030\001 \002(\014\022B\n\017qualifier_value\030\002 \003(\0132).Mutat" +
"Value.QualifierValue\032\203\001\n\016QualifierValue\022" + "ionProto.ColumnValue.QualifierValue\032\203\001\n\016" +
"\021\n\tqualifier\030\001 \001(\014\022\r\n\005value\030\002 \001(\014\022\021\n\ttim", "QualifierValue\022\021\n\tqualifier\030\001 \001(\014\022\r\n\005val",
"estamp\030\003 \001(\004\022.\n\013delete_type\030\004 \001(\0162\031.Muta" + "ue\030\002 \001(\014\022\021\n\ttimestamp\030\003 \001(\004\022.\n\013delete_ty" +
"tionProto.DeleteType\022\014\n\004tags\030\005 \001(\014\"W\n\nDu" + "pe\030\004 \001(\0162\031.MutationProto.DeleteType\022\014\n\004t" +
"rability\022\017\n\013USE_DEFAULT\020\000\022\014\n\010SKIP_WAL\020\001\022" + "ags\030\005 \001(\014\"W\n\nDurability\022\017\n\013USE_DEFAULT\020\000" +
"\r\n\tASYNC_WAL\020\002\022\014\n\010SYNC_WAL\020\003\022\r\n\tFSYNC_WA" + "\022\014\n\010SKIP_WAL\020\001\022\r\n\tASYNC_WAL\020\002\022\014\n\010SYNC_WA" +
"L\020\004\">\n\014MutationType\022\n\n\006APPEND\020\000\022\r\n\tINCRE" + "L\020\003\022\r\n\tFSYNC_WAL\020\004\">\n\014MutationType\022\n\n\006AP" +
"MENT\020\001\022\007\n\003PUT\020\002\022\n\n\006DELETE\020\003\"p\n\nDeleteTyp" + "PEND\020\000\022\r\n\tINCREMENT\020\001\022\007\n\003PUT\020\002\022\n\n\006DELETE" +
"e\022\026\n\022DELETE_ONE_VERSION\020\000\022\034\n\030DELETE_MULT" + "\020\003\"p\n\nDeleteType\022\026\n\022DELETE_ONE_VERSION\020\000" +
"IPLE_VERSIONS\020\001\022\021\n\rDELETE_FAMILY\020\002\022\031\n\025DE" + "\022\034\n\030DELETE_MULTIPLE_VERSIONS\020\001\022\021\n\rDELETE" +
"LETE_FAMILY_VERSION\020\003\"r\n\rMutateRequest\022 " + "_FAMILY\020\002\022\031\n\025DELETE_FAMILY_VERSION\020\003\"\207\001\n" +
"\n\006region\030\001 \002(\0132\020.RegionSpecifier\022 \n\010muta", "\rMutateRequest\022 \n\006region\030\001 \002(\0132\020.RegionS",
"tion\030\002 \002(\0132\016.MutationProto\022\035\n\tcondition\030" + "pecifier\022 \n\010mutation\030\002 \002(\0132\016.MutationPro" +
"\003 \001(\0132\n.Condition\"<\n\016MutateResponse\022\027\n\006r" + "to\022\035\n\tcondition\030\003 \001(\0132\n.Condition\022\023\n\013non" +
"esult\030\001 \001(\0132\007.Result\022\021\n\tprocessed\030\002 \001(\010\"" + "ce_group\030\004 \001(\004\"<\n\016MutateResponse\022\027\n\006resu" +
"\344\002\n\004Scan\022\027\n\006column\030\001 \003(\0132\007.Column\022!\n\tatt" + "lt\030\001 \001(\0132\007.Result\022\021\n\tprocessed\030\002 \001(\010\"\344\002\n" +
"ribute\030\002 \003(\0132\016.NameBytesPair\022\021\n\tstart_ro" + "\004Scan\022\027\n\006column\030\001 \003(\0132\007.Column\022!\n\tattrib" +
"w\030\003 \001(\014\022\020\n\010stop_row\030\004 \001(\014\022\027\n\006filter\030\005 \001(" + "ute\030\002 \003(\0132\016.NameBytesPair\022\021\n\tstart_row\030\003" +
"\0132\007.Filter\022\036\n\ntime_range\030\006 \001(\0132\n.TimeRan" + " \001(\014\022\020\n\010stop_row\030\004 \001(\014\022\027\n\006filter\030\005 \001(\0132\007" +
"ge\022\027\n\014max_versions\030\007 \001(\r:\0011\022\032\n\014cache_blo" + ".Filter\022\036\n\ntime_range\030\006 \001(\0132\n.TimeRange\022" +
"cks\030\010 \001(\010:\004true\022\022\n\nbatch_size\030\t \001(\r\022\027\n\017m" + "\027\n\014max_versions\030\007 \001(\r:\0011\022\032\n\014cache_blocks" +
"ax_result_size\030\n \001(\004\022\023\n\013store_limit\030\013 \001(", "\030\010 \001(\010:\004true\022\022\n\nbatch_size\030\t \001(\r\022\027\n\017max_",
"\r\022\024\n\014store_offset\030\014 \001(\r\022&\n\036load_column_f" + "result_size\030\n \001(\004\022\023\n\013store_limit\030\013 \001(\r\022\024" +
"amilies_on_demand\030\r \001(\010\022\r\n\005small\030\016 \001(\010\"\236" + "\n\014store_offset\030\014 \001(\r\022&\n\036load_column_fami" +
"\001\n\013ScanRequest\022 \n\006region\030\001 \001(\0132\020.RegionS" + "lies_on_demand\030\r \001(\010\022\r\n\005small\030\016 \001(\010\"\236\001\n\013" +
"pecifier\022\023\n\004scan\030\002 \001(\0132\005.Scan\022\022\n\nscanner" + "ScanRequest\022 \n\006region\030\001 \001(\0132\020.RegionSpec" +
"_id\030\003 \001(\004\022\026\n\016number_of_rows\030\004 \001(\r\022\025\n\rclo" + "ifier\022\023\n\004scan\030\002 \001(\0132\005.Scan\022\022\n\nscanner_id" +
"se_scanner\030\005 \001(\010\022\025\n\rnext_call_seq\030\006 \001(\004\"" + "\030\003 \001(\004\022\026\n\016number_of_rows\030\004 \001(\r\022\025\n\rclose_" +
"y\n\014ScanResponse\022\030\n\020cells_per_result\030\001 \003(" + "scanner\030\005 \001(\010\022\025\n\rnext_call_seq\030\006 \001(\004\"y\n\014" +
"\r\022\022\n\nscanner_id\030\002 \001(\004\022\024\n\014more_results\030\003 " + "ScanResponse\022\030\n\020cells_per_result\030\001 \003(\r\022\022" +
"\001(\010\022\013\n\003ttl\030\004 \001(\r\022\030\n\007results\030\005 \003(\0132\007.Resu" + "\n\nscanner_id\030\002 \001(\004\022\024\n\014more_results\030\003 \001(\010" +
"lt\"\263\001\n\024BulkLoadHFileRequest\022 \n\006region\030\001 ", "\022\013\n\003ttl\030\004 \001(\r\022\030\n\007results\030\005 \003(\0132\007.Result\"",
"\002(\0132\020.RegionSpecifier\0225\n\013family_path\030\002 \003" + "\263\001\n\024BulkLoadHFileRequest\022 \n\006region\030\001 \002(\013" +
"(\0132 .BulkLoadHFileRequest.FamilyPath\022\026\n\016" + "2\020.RegionSpecifier\0225\n\013family_path\030\002 \003(\0132" +
"assign_seq_num\030\003 \001(\010\032*\n\nFamilyPath\022\016\n\006fa" + " .BulkLoadHFileRequest.FamilyPath\022\026\n\016ass" +
"mily\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFil" + "ign_seq_num\030\003 \001(\010\032*\n\nFamilyPath\022\016\n\006famil" +
"eResponse\022\016\n\006loaded\030\001 \002(\010\"a\n\026Coprocessor" + "y\030\001 \002(\014\022\014\n\004path\030\002 \002(\t\"\'\n\025BulkLoadHFileRe" +
"ServiceCall\022\013\n\003row\030\001 \002(\014\022\024\n\014service_name" + "sponse\022\016\n\006loaded\030\001 \002(\010\"a\n\026CoprocessorSer" +
"\030\002 \002(\t\022\023\n\013method_name\030\003 \002(\t\022\017\n\007request\030\004" + "viceCall\022\013\n\003row\030\001 \002(\014\022\024\n\014service_name\030\002 " +
" \002(\014\"d\n\031CoprocessorServiceRequest\022 \n\006reg" + "\002(\t\022\023\n\013method_name\030\003 \002(\t\022\017\n\007request\030\004 \002(" +
"ion\030\001 \002(\0132\020.RegionSpecifier\022%\n\004call\030\002 \002(" + "\014\"d\n\031CoprocessorServiceRequest\022 \n\006region" +
"\0132\027.CoprocessorServiceCall\"]\n\032Coprocesso", "\030\001 \002(\0132\020.RegionSpecifier\022%\n\004call\030\002 \002(\0132\027",
"rServiceResponse\022 \n\006region\030\001 \002(\0132\020.Regio" + ".CoprocessorServiceCall\"]\n\032CoprocessorSe" +
"nSpecifier\022\035\n\005value\030\002 \002(\0132\016.NameBytesPai" + "rviceResponse\022 \n\006region\030\001 \002(\0132\020.RegionSp" +
"r\"L\n\006Action\022\r\n\005index\030\001 \001(\r\022 \n\010mutation\030\002" + "ecifier\022\035\n\005value\030\002 \002(\0132\016.NameBytesPair\"L" +
" \001(\0132\016.MutationProto\022\021\n\003get\030\003 \001(\0132\004.Get\"" + "\n\006Action\022\r\n\005index\030\001 \001(\r\022 \n\010mutation\030\002 \001(" +
"Y\n\014RegionAction\022 \n\006region\030\001 \002(\0132\020.Region" + "\0132\016.MutationProto\022\021\n\003get\030\003 \001(\0132\004.Get\"Y\n\014" +
"Specifier\022\016\n\006atomic\030\002 \001(\010\022\027\n\006action\030\003 \003(" + "RegionAction\022 \n\006region\030\001 \002(\0132\020.RegionSpe" +
"\0132\007.Action\"^\n\021ResultOrException\022\r\n\005index" + "cifier\022\016\n\006atomic\030\002 \001(\010\022\027\n\006action\030\003 \003(\0132\007" +
"\030\001 \001(\r\022\027\n\006result\030\002 \001(\0132\007.Result\022!\n\texcep" + ".Action\"^\n\021ResultOrException\022\r\n\005index\030\001 " +
"tion\030\003 \001(\0132\016.NameBytesPair\"f\n\022RegionActi" + "\001(\r\022\027\n\006result\030\002 \001(\0132\007.Result\022!\n\texceptio" +
"onResult\022-\n\021resultOrException\030\001 \003(\0132\022.Re", "n\030\003 \001(\0132\016.NameBytesPair\"f\n\022RegionActionR",
"sultOrException\022!\n\texception\030\002 \001(\0132\016.Nam" + "esult\022-\n\021resultOrException\030\001 \003(\0132\022.Resul" +
"eBytesPair\"3\n\014MultiRequest\022#\n\014regionActi" + "tOrException\022!\n\texception\030\002 \001(\0132\016.NameBy" +
"on\030\001 \003(\0132\r.RegionAction\"@\n\rMultiResponse" + "tesPair\"G\n\014MultiRequest\022#\n\014regionAction\030" +
"\022/\n\022regionActionResult\030\001 \003(\0132\023.RegionAct" + "\001 \003(\0132\r.RegionAction\022\022\n\nnonceGroup\030\002 \001(\004" +
"ionResult2\261\002\n\rClientService\022 \n\003Get\022\013.Get" + "\"@\n\rMultiResponse\022/\n\022regionActionResult\030" +
"Request\032\014.GetResponse\022)\n\006Mutate\022\016.Mutate" + "\001 \003(\0132\023.RegionActionResult2\261\002\n\rClientSer" +
"Request\032\017.MutateResponse\022#\n\004Scan\022\014.ScanR" + "vice\022 \n\003Get\022\013.GetRequest\032\014.GetResponse\022)" +
"equest\032\r.ScanResponse\022>\n\rBulkLoadHFile\022\025" + "\n\006Mutate\022\016.MutateRequest\032\017.MutateRespons" +
".BulkLoadHFileRequest\032\026.BulkLoadHFileRes" + "e\022#\n\004Scan\022\014.ScanRequest\032\r.ScanResponse\022>" +
"ponse\022F\n\013ExecService\022\032.CoprocessorServic", "\n\rBulkLoadHFile\022\025.BulkLoadHFileRequest\032\026",
"eRequest\032\033.CoprocessorServiceResponse\022&\n" + ".BulkLoadHFileResponse\022F\n\013ExecService\022\032." +
"\005Multi\022\r.MultiRequest\032\016.MultiResponseBB\n" + "CoprocessorServiceRequest\032\033.CoprocessorS" +
"*org.apache.hadoop.hbase.protobuf.genera" + "erviceResponse\022&\n\005Multi\022\r.MultiRequest\032\016" +
"tedB\014ClientProtosH\001\210\001\001\240\001\001" ".MultiResponseBB\n*org.apache.hadoop.hbas" +
"e.protobuf.generatedB\014ClientProtosH\001\210\001\001\240" +
"\001\001"
}; };
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -27750,7 +28025,7 @@ public final class ClientProtos {
internal_static_MutationProto_fieldAccessorTable = new internal_static_MutationProto_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable( com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_MutationProto_descriptor, 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_ColumnValue_descriptor =
internal_static_MutationProto_descriptor.getNestedTypes().get(0); internal_static_MutationProto_descriptor.getNestedTypes().get(0);
internal_static_MutationProto_ColumnValue_fieldAccessorTable = new internal_static_MutationProto_ColumnValue_fieldAccessorTable = new
@ -27768,7 +28043,7 @@ public final class ClientProtos {
internal_static_MutateRequest_fieldAccessorTable = new internal_static_MutateRequest_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable( com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_MutateRequest_descriptor, internal_static_MutateRequest_descriptor,
new java.lang.String[] { "Region", "Mutation", "Condition", }); new java.lang.String[] { "Region", "Mutation", "Condition", "NonceGroup", });
internal_static_MutateResponse_descriptor = internal_static_MutateResponse_descriptor =
getDescriptor().getMessageTypes().get(8); getDescriptor().getMessageTypes().get(8);
internal_static_MutateResponse_fieldAccessorTable = new internal_static_MutateResponse_fieldAccessorTable = new
@ -27858,7 +28133,7 @@ public final class ClientProtos {
internal_static_MultiRequest_fieldAccessorTable = new internal_static_MultiRequest_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable( com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_MultiRequest_descriptor, internal_static_MultiRequest_descriptor,
new java.lang.String[] { "RegionAction", }); new java.lang.String[] { "RegionAction", "NonceGroup", });
internal_static_MultiResponse_descriptor = internal_static_MultiResponse_descriptor =
getDescriptor().getMessageTypes().get(22); getDescriptor().getMessageTypes().get(22);
internal_static_MultiResponse_fieldAccessorTable = new internal_static_MultiResponse_fieldAccessorTable = new

View File

@ -711,6 +711,26 @@ public final class MultiRowMutationProtos {
*/ */
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getMutationRequestOrBuilder( org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProtoOrBuilder getMutationRequestOrBuilder(
int index); 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} * 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)); mutationRequest_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.PARSER, extensionRegistry));
break; break;
} }
case 16: {
bitField0_ |= 0x00000001;
nonceGroup_ = input.readUInt64();
break;
}
case 24: {
bitField0_ |= 0x00000002;
nonce_ = input.readUInt64();
break;
}
} }
} }
} catch (com.google.protobuf.InvalidProtocolBufferException e) { } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@ -813,6 +843,7 @@ public final class MultiRowMutationProtos {
return PARSER; return PARSER;
} }
private int bitField0_;
// repeated .MutationProto mutation_request = 1; // repeated .MutationProto mutation_request = 1;
public static final int MUTATION_REQUEST_FIELD_NUMBER = 1; public static final int MUTATION_REQUEST_FIELD_NUMBER = 1;
private java.util.List<org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto> mutationRequest_; 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); 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() { private void initFields() {
mutationRequest_ = java.util.Collections.emptyList(); mutationRequest_ = java.util.Collections.emptyList();
nonceGroup_ = 0L;
nonce_ = 0L;
} }
private byte memoizedIsInitialized = -1; private byte memoizedIsInitialized = -1;
public final boolean isInitialized() { public final boolean isInitialized() {
@ -873,6 +938,12 @@ public final class MultiRowMutationProtos {
for (int i = 0; i < mutationRequest_.size(); i++) { for (int i = 0; i < mutationRequest_.size(); i++) {
output.writeMessage(1, mutationRequest_.get(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); getUnknownFields().writeTo(output);
} }
@ -886,6 +957,14 @@ public final class MultiRowMutationProtos {
size += com.google.protobuf.CodedOutputStream size += com.google.protobuf.CodedOutputStream
.computeMessageSize(1, mutationRequest_.get(i)); .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(); size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size; memoizedSerializedSize = size;
return size; return size;
@ -911,6 +990,16 @@ public final class MultiRowMutationProtos {
boolean result = true; boolean result = true;
result = result && getMutationRequestList() result = result && getMutationRequestList()
.equals(other.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 && result = result &&
getUnknownFields().equals(other.getUnknownFields()); getUnknownFields().equals(other.getUnknownFields());
return result; return result;
@ -928,6 +1017,14 @@ public final class MultiRowMutationProtos {
hash = (37 * hash) + MUTATION_REQUEST_FIELD_NUMBER; hash = (37 * hash) + MUTATION_REQUEST_FIELD_NUMBER;
hash = (53 * hash) + getMutationRequestList().hashCode(); 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(); hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash; memoizedHashCode = hash;
return hash; return hash;
@ -1044,6 +1141,10 @@ public final class MultiRowMutationProtos {
} else { } else {
mutationRequestBuilder_.clear(); mutationRequestBuilder_.clear();
} }
nonceGroup_ = 0L;
bitField0_ = (bitField0_ & ~0x00000002);
nonce_ = 0L;
bitField0_ = (bitField0_ & ~0x00000004);
return this; return this;
} }
@ -1071,6 +1172,7 @@ public final class MultiRowMutationProtos {
public org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest buildPartial() { 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); org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest result = new org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest(this);
int from_bitField0_ = bitField0_; int from_bitField0_ = bitField0_;
int to_bitField0_ = 0;
if (mutationRequestBuilder_ == null) { if (mutationRequestBuilder_ == null) {
if (((bitField0_ & 0x00000001) == 0x00000001)) { if (((bitField0_ & 0x00000001) == 0x00000001)) {
mutationRequest_ = java.util.Collections.unmodifiableList(mutationRequest_); mutationRequest_ = java.util.Collections.unmodifiableList(mutationRequest_);
@ -1080,6 +1182,15 @@ public final class MultiRowMutationProtos {
} else { } else {
result.mutationRequest_ = mutationRequestBuilder_.build(); 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(); onBuilt();
return result; 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()); this.mergeUnknownFields(other.getUnknownFields());
return this; return this;
} }
@ -1394,6 +1511,72 @@ public final class MultiRowMutationProtos {
return mutationRequestBuilder_; 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) // @@protoc_insertion_point(builder_scope:MutateRowsRequest)
} }
@ -2006,13 +2189,14 @@ public final class MultiRowMutationProtos {
java.lang.String[] descriptorData = { java.lang.String[] descriptorData = {
"\n\026MultiRowMutation.proto\032\014Client.proto\"\"" + "\n\026MultiRowMutation.proto\032\014Client.proto\"\"" +
"\n MultiRowMutationProcessorRequest\"#\n!Mu" + "\n MultiRowMutationProcessorRequest\"#\n!Mu" +
"ltiRowMutationProcessorResponse\"=\n\021Mutat" + "ltiRowMutationProcessorResponse\"a\n\021Mutat" +
"eRowsRequest\022(\n\020mutation_request\030\001 \003(\0132\016" + "eRowsRequest\022(\n\020mutation_request\030\001 \003(\0132\016" +
".MutationProto\"\024\n\022MutateRowsResponse2P\n\027" + ".MutationProto\022\023\n\013nonce_group\030\002 \001(\004\022\r\n\005n" +
"MultiRowMutationService\0225\n\nMutateRows\022\022." + "once\030\003 \001(\004\"\024\n\022MutateRowsResponse2P\n\027Mult" +
"MutateRowsRequest\032\023.MutateRowsResponseBL" + "iRowMutationService\0225\n\nMutateRows\022\022.Muta" +
"\n*org.apache.hadoop.hbase.protobuf.gener" + "teRowsRequest\032\023.MutateRowsResponseBL\n*or" +
"atedB\026MultiRowMutationProtosH\001\210\001\001\240\001\001" "g.apache.hadoop.hbase.protobuf.generated" +
"B\026MultiRowMutationProtosH\001\210\001\001\240\001\001"
}; };
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -2036,7 +2220,7 @@ public final class MultiRowMutationProtos {
internal_static_MutateRowsRequest_fieldAccessorTable = new internal_static_MutateRowsRequest_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable( com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_MutateRowsRequest_descriptor, internal_static_MutateRowsRequest_descriptor,
new java.lang.String[] { "MutationRequest", }); new java.lang.String[] { "MutationRequest", "NonceGroup", "Nonce", });
internal_static_MutateRowsResponse_descriptor = internal_static_MutateRowsResponse_descriptor =
getDescriptor().getMessageTypes().get(3); getDescriptor().getMessageTypes().get(3);
internal_static_MutateRowsResponse_fieldAccessorTable = new internal_static_MutateRowsResponse_fieldAccessorTable = new

View File

@ -50,6 +50,26 @@ public final class RowProcessorProtos {
* <code>optional bytes row_processor_initializer_message = 3;</code> * <code>optional bytes row_processor_initializer_message = 3;</code>
*/ */
com.google.protobuf.ByteString getRowProcessorInitializerMessage(); 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} * Protobuf type {@code ProcessRequest}
@ -117,6 +137,16 @@ public final class RowProcessorProtos {
rowProcessorInitializerMessage_ = input.readBytes(); rowProcessorInitializerMessage_ = input.readBytes();
break; break;
} }
case 32: {
bitField0_ |= 0x00000008;
nonceGroup_ = input.readUInt64();
break;
}
case 40: {
bitField0_ |= 0x00000010;
nonce_ = input.readUInt64();
break;
}
} }
} }
} catch (com.google.protobuf.InvalidProtocolBufferException e) { } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@ -259,10 +289,44 @@ public final class RowProcessorProtos {
return rowProcessorInitializerMessage_; 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() { private void initFields() {
rowProcessorClassName_ = ""; rowProcessorClassName_ = "";
rowProcessorInitializerMessageName_ = ""; rowProcessorInitializerMessageName_ = "";
rowProcessorInitializerMessage_ = com.google.protobuf.ByteString.EMPTY; rowProcessorInitializerMessage_ = com.google.protobuf.ByteString.EMPTY;
nonceGroup_ = 0L;
nonce_ = 0L;
} }
private byte memoizedIsInitialized = -1; private byte memoizedIsInitialized = -1;
public final boolean isInitialized() { public final boolean isInitialized() {
@ -289,6 +353,12 @@ public final class RowProcessorProtos {
if (((bitField0_ & 0x00000004) == 0x00000004)) { if (((bitField0_ & 0x00000004) == 0x00000004)) {
output.writeBytes(3, rowProcessorInitializerMessage_); output.writeBytes(3, rowProcessorInitializerMessage_);
} }
if (((bitField0_ & 0x00000008) == 0x00000008)) {
output.writeUInt64(4, nonceGroup_);
}
if (((bitField0_ & 0x00000010) == 0x00000010)) {
output.writeUInt64(5, nonce_);
}
getUnknownFields().writeTo(output); getUnknownFields().writeTo(output);
} }
@ -310,6 +380,14 @@ public final class RowProcessorProtos {
size += com.google.protobuf.CodedOutputStream size += com.google.protobuf.CodedOutputStream
.computeBytesSize(3, rowProcessorInitializerMessage_); .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(); size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size; memoizedSerializedSize = size;
return size; return size;
@ -348,6 +426,16 @@ public final class RowProcessorProtos {
result = result && getRowProcessorInitializerMessage() result = result && getRowProcessorInitializerMessage()
.equals(other.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 && result = result &&
getUnknownFields().equals(other.getUnknownFields()); getUnknownFields().equals(other.getUnknownFields());
return result; return result;
@ -373,6 +461,14 @@ public final class RowProcessorProtos {
hash = (37 * hash) + ROW_PROCESSOR_INITIALIZER_MESSAGE_FIELD_NUMBER; hash = (37 * hash) + ROW_PROCESSOR_INITIALIZER_MESSAGE_FIELD_NUMBER;
hash = (53 * hash) + getRowProcessorInitializerMessage().hashCode(); 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(); hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash; memoizedHashCode = hash;
return hash; return hash;
@ -488,6 +584,10 @@ public final class RowProcessorProtos {
bitField0_ = (bitField0_ & ~0x00000002); bitField0_ = (bitField0_ & ~0x00000002);
rowProcessorInitializerMessage_ = com.google.protobuf.ByteString.EMPTY; rowProcessorInitializerMessage_ = com.google.protobuf.ByteString.EMPTY;
bitField0_ = (bitField0_ & ~0x00000004); bitField0_ = (bitField0_ & ~0x00000004);
nonceGroup_ = 0L;
bitField0_ = (bitField0_ & ~0x00000008);
nonce_ = 0L;
bitField0_ = (bitField0_ & ~0x00000010);
return this; return this;
} }
@ -528,6 +628,14 @@ public final class RowProcessorProtos {
to_bitField0_ |= 0x00000004; to_bitField0_ |= 0x00000004;
} }
result.rowProcessorInitializerMessage_ = rowProcessorInitializerMessage_; 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_; result.bitField0_ = to_bitField0_;
onBuilt(); onBuilt();
return result; return result;
@ -557,6 +665,12 @@ public final class RowProcessorProtos {
if (other.hasRowProcessorInitializerMessage()) { if (other.hasRowProcessorInitializerMessage()) {
setRowProcessorInitializerMessage(other.getRowProcessorInitializerMessage()); setRowProcessorInitializerMessage(other.getRowProcessorInitializerMessage());
} }
if (other.hasNonceGroup()) {
setNonceGroup(other.getNonceGroup());
}
if (other.hasNonce()) {
setNonce(other.getNonce());
}
this.mergeUnknownFields(other.getUnknownFields()); this.mergeUnknownFields(other.getUnknownFields());
return this; return this;
} }
@ -772,6 +886,72 @@ public final class RowProcessorProtos {
return this; 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) // @@protoc_insertion_point(builder_scope:ProcessRequest)
} }
@ -1479,15 +1659,16 @@ public final class RowProcessorProtos {
descriptor; descriptor;
static { static {
java.lang.String[] descriptorData = { 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" + " \n\030row_processor_class_name\030\001 \002(\t\022.\n&row" +
"_processor_initializer_message_name\030\002 \001(" + "_processor_initializer_message_name\030\002 \001(" +
"\t\022)\n!row_processor_initializer_message\030\003" + "\t\022)\n!row_processor_initializer_message\030\003" +
" \001(\014\"/\n\017ProcessResponse\022\034\n\024row_processor" + " \001(\014\022\023\n\013nonce_group\030\004 \001(\004\022\r\n\005nonce\030\005 \001(\004" +
"_result\030\001 \002(\0142C\n\023RowProcessorService\022,\n\007" + "\"/\n\017ProcessResponse\022\034\n\024row_processor_res" +
"Process\022\017.ProcessRequest\032\020.ProcessRespon" + "ult\030\001 \002(\0142C\n\023RowProcessorService\022,\n\007Proc" +
"seBH\n*org.apache.hadoop.hbase.protobuf.g" + "ess\022\017.ProcessRequest\032\020.ProcessResponseBH" +
"eneratedB\022RowProcessorProtosH\001\210\001\001\240\001\001" "\n*org.apache.hadoop.hbase.protobuf.gener" +
"atedB\022RowProcessorProtosH\001\210\001\001\240\001\001"
}; };
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -1499,7 +1680,7 @@ public final class RowProcessorProtos {
internal_static_ProcessRequest_fieldAccessorTable = new internal_static_ProcessRequest_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable( com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_ProcessRequest_descriptor, internal_static_ProcessRequest_descriptor,
new java.lang.String[] { "RowProcessorClassName", "RowProcessorInitializerMessageName", "RowProcessorInitializerMessage", }); new java.lang.String[] { "RowProcessorClassName", "RowProcessorInitializerMessageName", "RowProcessorInitializerMessage", "NonceGroup", "Nonce", });
internal_static_ProcessResponse_descriptor = internal_static_ProcessResponse_descriptor =
getDescriptor().getMessageTypes().get(1); getDescriptor().getMessageTypes().get(1);
internal_static_ProcessResponse_fieldAccessorTable = new internal_static_ProcessResponse_fieldAccessorTable = new

View File

@ -694,6 +694,26 @@ public final class WALProtos {
*/ */
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.UUIDOrBuilder getClusterIdsOrBuilder( org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.UUIDOrBuilder getClusterIdsOrBuilder(
int index); 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} * 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)); clusterIds_.add(input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.UUID.PARSER, extensionRegistry));
break; break;
} }
case 72: {
bitField0_ |= 0x00000040;
nonceGroup_ = input.readUInt64();
break;
}
case 80: {
bitField0_ |= 0x00000080;
nonce_ = input.readUInt64();
break;
}
} }
} }
} catch (com.google.protobuf.InvalidProtocolBufferException e) { } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@ -1078,6 +1108,38 @@ public final class WALProtos {
return clusterIds_.get(index); 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() { private void initFields() {
encodedRegionName_ = com.google.protobuf.ByteString.EMPTY; encodedRegionName_ = com.google.protobuf.ByteString.EMPTY;
tableName_ = com.google.protobuf.ByteString.EMPTY; tableName_ = com.google.protobuf.ByteString.EMPTY;
@ -1087,6 +1149,8 @@ public final class WALProtos {
scopes_ = java.util.Collections.emptyList(); scopes_ = java.util.Collections.emptyList();
followingKvCount_ = 0; followingKvCount_ = 0;
clusterIds_ = java.util.Collections.emptyList(); clusterIds_ = java.util.Collections.emptyList();
nonceGroup_ = 0L;
nonce_ = 0L;
} }
private byte memoizedIsInitialized = -1; private byte memoizedIsInitialized = -1;
public final boolean isInitialized() { public final boolean isInitialized() {
@ -1158,6 +1222,12 @@ public final class WALProtos {
for (int i = 0; i < clusterIds_.size(); i++) { for (int i = 0; i < clusterIds_.size(); i++) {
output.writeMessage(8, clusterIds_.get(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); getUnknownFields().writeTo(output);
} }
@ -1199,6 +1269,14 @@ public final class WALProtos {
size += com.google.protobuf.CodedOutputStream size += com.google.protobuf.CodedOutputStream
.computeMessageSize(8, clusterIds_.get(i)); .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(); size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size; memoizedSerializedSize = size;
return size; return size;
@ -1256,6 +1334,16 @@ public final class WALProtos {
} }
result = result && getClusterIdsList() result = result && getClusterIdsList()
.equals(other.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 && result = result &&
getUnknownFields().equals(other.getUnknownFields()); getUnknownFields().equals(other.getUnknownFields());
return result; return result;
@ -1301,6 +1389,14 @@ public final class WALProtos {
hash = (37 * hash) + CLUSTER_IDS_FIELD_NUMBER; hash = (37 * hash) + CLUSTER_IDS_FIELD_NUMBER;
hash = (53 * hash) + getClusterIdsList().hashCode(); 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(); hash = (29 * hash) + getUnknownFields().hashCode();
memoizedHashCode = hash; memoizedHashCode = hash;
return hash; return hash;
@ -1445,6 +1541,10 @@ public final class WALProtos {
} else { } else {
clusterIdsBuilder_.clear(); clusterIdsBuilder_.clear();
} }
nonceGroup_ = 0L;
bitField0_ = (bitField0_ & ~0x00000100);
nonce_ = 0L;
bitField0_ = (bitField0_ & ~0x00000200);
return this; return this;
} }
@ -1519,6 +1619,14 @@ public final class WALProtos {
} else { } else {
result.clusterIds_ = clusterIdsBuilder_.build(); 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_; result.bitField0_ = to_bitField0_;
onBuilt(); onBuilt();
return result; 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()); this.mergeUnknownFields(other.getUnknownFields());
return this; return this;
} }
@ -2614,6 +2728,72 @@ public final class WALProtos {
return clusterIdsBuilder_; 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) // @@protoc_insertion_point(builder_scope:WALKey)
} }
@ -4812,22 +4992,23 @@ public final class WALProtos {
static { static {
java.lang.String[] descriptorData = { java.lang.String[] descriptorData = {
"\n\tWAL.proto\032\013HBase.proto\"$\n\tWALHeader\022\027\n" + "\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" + "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" + "(\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" + "_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" + "\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(" + "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" + "\0132\005.UUID\022\022\n\nnonceGroup\030\t \001(\004\022\r\n\005nonce\030\n " +
"\036\n\nscope_type\030\002 \002(\0162\n.ScopeType\"\251\001\n\024Comp" + "\001(\004\"=\n\013FamilyScope\022\016\n\006family\030\001 \002(\014\022\036\n\nsc" +
"actionDescriptor\022\022\n\ntable_name\030\001 \002(\014\022\033\n\023", "ope_type\030\002 \002(\0162\n.ScopeType\"\251\001\n\024Compactio",
"encoded_region_name\030\002 \002(\014\022\023\n\013family_name" + "nDescriptor\022\022\n\ntable_name\030\001 \002(\014\022\033\n\023encod" +
"\030\003 \002(\014\022\030\n\020compaction_input\030\004 \003(\t\022\031\n\021comp" + "ed_region_name\030\002 \002(\014\022\023\n\013family_name\030\003 \002(" +
"action_output\030\005 \003(\t\022\026\n\016store_home_dir\030\006 " + "\014\022\030\n\020compaction_input\030\004 \003(\t\022\031\n\021compactio" +
"\002(\t\"\014\n\nWALTrailer*F\n\tScopeType\022\033\n\027REPLIC" + "n_output\030\005 \003(\t\022\026\n\016store_home_dir\030\006 \002(\t\"\014" +
"ATION_SCOPE_LOCAL\020\000\022\034\n\030REPLICATION_SCOPE" + "\n\nWALTrailer*F\n\tScopeType\022\033\n\027REPLICATION" +
"_GLOBAL\020\001B?\n*org.apache.hadoop.hbase.pro" + "_SCOPE_LOCAL\020\000\022\034\n\030REPLICATION_SCOPE_GLOB" +
"tobuf.generatedB\tWALProtosH\001\210\001\000\240\001\001" "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 = com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@ -4845,7 +5026,7 @@ public final class WALProtos {
internal_static_WALKey_fieldAccessorTable = new internal_static_WALKey_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable( com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_WALKey_descriptor, 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 = internal_static_FamilyScope_descriptor =
getDescriptor().getMessageTypes().get(2); getDescriptor().getMessageTypes().get(2);
internal_static_FamilyScope_fieldAccessorTable = new 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. // 'cell' field above which is non-null when the cells are pb'd.
optional int32 associated_cell_count = 8; optional int32 associated_cell_count = 8;
optional uint64 nonce = 9;
enum Durability { enum Durability {
USE_DEFAULT = 0; USE_DEFAULT = 0;
SKIP_WAL = 1; SKIP_WAL = 1;
@ -182,6 +184,7 @@ message MutateRequest {
required RegionSpecifier region = 1; required RegionSpecifier region = 1;
required MutationProto mutation = 2; required MutationProto mutation = 2;
optional Condition condition = 3; optional Condition condition = 3;
optional uint64 nonce_group = 4;
} }
message MutateResponse { message MutateResponse {
@ -345,6 +348,7 @@ message RegionActionResult {
*/ */
message MultiRequest { message MultiRequest {
repeated RegionAction regionAction = 1; repeated RegionAction regionAction = 1;
optional uint64 nonceGroup = 2;
} }
message MultiResponse { message MultiResponse {

View File

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

View File

@ -30,6 +30,8 @@ message ProcessRequest {
required string row_processor_class_name = 1; required string row_processor_class_name = 1;
optional string row_processor_initializer_message_name = 2; optional string row_processor_initializer_message_name = 2;
optional bytes row_processor_initializer_message = 3; optional bytes row_processor_initializer_message = 3;
optional uint64 nonce_group = 4;
optional uint64 nonce = 5;
} }
message ProcessResponse { message ProcessResponse {

View File

@ -43,11 +43,16 @@ message WALKey {
repeated FamilyScope scopes = 6; repeated FamilyScope scopes = 6;
optional uint32 following_kv_count = 7; optional uint32 following_kv_count = 7;
/* /*
This field contains the list of clusters that have This field contains the list of clusters that have
consumed the change consumed the change
*/ */
repeated UUID cluster_ids = 8; repeated UUID cluster_ids = 8;
optional uint64 nonceGroup = 9;
optional uint64 nonce = 10;
/* /*
optional CustomEntryType custom_entry_type = 9; optional CustomEntryType custom_entry_type = 9;

View File

@ -380,4 +380,9 @@ public class CoprocessorHConnection implements HConnection {
public boolean isDeadServer(ServerName serverName) { public boolean isDeadServer(ServerName serverName) {
return delegate.isDeadServer(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.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseInterfaceAudience; 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.ResponseConverter;
import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessRequest; import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessRequest;
import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessResponse; import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessResponse;
@ -68,7 +69,9 @@ extends RowProcessorService implements CoprocessorService, Coprocessor {
try { try {
RowProcessor<S,T> processor = constructRowProcessorFromRequest(request); RowProcessor<S,T> processor = constructRowProcessorFromRequest(request);
HRegion region = env.getRegion(); 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(); T result = processor.getResult();
ProcessResponse.Builder b = ProcessResponse.newBuilder(); ProcessResponse.Builder b = ProcessResponse.newBuilder();
b.setRowProcessorResult(result.toByteString()); 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.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment; import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
@ -110,7 +111,9 @@ CoprocessorService, Coprocessor {
rowsToLock.add(m.getRow()); rowsToLock.add(m.getRow());
} }
// call utility method on region // 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) { } catch (IOException e) {
ResponseConverter.setControllerException(controller, 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.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.CellScanner;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.io.SizedCellScanner; import org.apache.hadoop.hbase.io.SizedCellScanner;
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
@ -86,6 +87,7 @@ public class ReplicationProtbufUtil {
HBaseProtos.UUID.Builder uuidBuilder = HBaseProtos.UUID.newBuilder(); HBaseProtos.UUID.Builder uuidBuilder = HBaseProtos.UUID.newBuilder();
for (HLog.Entry entry: entries) { for (HLog.Entry entry: entries) {
entryBuilder.clear(); entryBuilder.clear();
// TODO: this duplicates a lot in HLogKey#getBuilder
WALProtos.WALKey.Builder keyBuilder = entryBuilder.getKeyBuilder(); WALProtos.WALKey.Builder keyBuilder = entryBuilder.getKeyBuilder();
HLogKey key = entry.getKey(); HLogKey key = entry.getKey();
keyBuilder.setEncodedRegionName( keyBuilder.setEncodedRegionName(
@ -93,6 +95,12 @@ public class ReplicationProtbufUtil {
keyBuilder.setTableName(ZeroCopyLiteralByteString.wrap(key.getTablename().getName())); keyBuilder.setTableName(ZeroCopyLiteralByteString.wrap(key.getTablename().getName()));
keyBuilder.setLogSequenceNumber(key.getLogSeqNum()); keyBuilder.setLogSequenceNumber(key.getLogSeqNum());
keyBuilder.setWriteTime(key.getWriteTime()); 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()) { for(UUID clusterId : key.getClusterIds()) {
uuidBuilder.setLeastSigBits(clusterId.getLeastSignificantBits()); uuidBuilder.setLeastSigBits(clusterId.getLeastSignificantBits());
uuidBuilder.setMostSigBits(clusterId.getMostSignificantBits()); 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.Cell;
import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.CompoundConfiguration; import org.apache.hadoop.hbase.CompoundConfiguration;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.DroppedSnapshotException; import org.apache.hadoop.hbase.DroppedSnapshotException;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HColumnDescriptor; 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.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory; import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
import org.apache.hadoop.hbase.regionserver.wal.HLogKey; 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.HLogUtil;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; 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 * accumulating status codes and tracking the index at which processing
* is proceeding. * is proceeding.
*/ */
private static class BatchOperationInProgress<T> { private abstract static class BatchOperationInProgress<T> {
T[] operations; T[] operations;
int nextIndexToProcess = 0; int nextIndexToProcess = 0;
OperationStatus[] retCodeDetails; OperationStatus[] retCodeDetails;
@ -1910,11 +1913,84 @@ public class HRegion implements HeapSize { // , Writable{
Arrays.fill(this.retCodeDetails, OperationStatus.NOT_RUN); 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() { public boolean isDone() {
return nextIndexToProcess == operations.length; return nextIndexToProcess == operations.length;
} }
} }
private static class MutationBatch extends BatchOperationInProgress<Mutation> {
private long nonceGroup;
private long nonce;
public MutationBatch(Mutation[] operations, long nonceGroup, long nonce) {
super(operations);
this.nonceGroup = nonceGroup;
this.nonce = nonce;
}
public Mutation getMutation(int index) {
return this.operations[index];
}
@Override
public long getNonceGroup(int index) {
return nonceGroup;
}
@Override
public long getNonce(int index) {
return nonce;
}
@Override
public Mutation[] getMutationsForCoprocs() {
return this.operations;
}
@Override
public boolean isInReplay() {
return false;
}
}
private static class ReplayBatch extends BatchOperationInProgress<HLogSplitter.MutationReplay> {
public ReplayBatch(MutationReplay[] operations) {
super(operations);
}
@Override
public Mutation getMutation(int index) {
return this.operations[index].mutation;
}
@Override
public long getNonceGroup(int index) {
return this.operations[index].nonceGroup;
}
@Override
public long getNonce(int index) {
return this.operations[index].nonce;
}
@Override
public Mutation[] getMutationsForCoprocs() {
assert false;
throw new RuntimeException("Should not be called for replay batch");
}
@Override
public boolean isInReplay() {
return true;
}
}
/** /**
* Perform a batch of mutations. * Perform a batch of mutations.
* It supports only Put and Delete mutations and will ignore other types passed. * It supports only Put and Delete mutations and will ignore other types passed.
@ -1923,8 +1999,28 @@ public class HRegion implements HeapSize { // , Writable{
* OperationStatusCode and the exceptionMessage if any. * OperationStatusCode and the exceptionMessage if any.
* @throws IOException * @throws IOException
*/ */
public OperationStatus[] batchMutate(
Mutation[] mutations, long nonceGroup, long nonce) throws IOException {
// As it stands, this is used for 3 things
// * batchMutate with single mutation - put/delete, separate or from checkAndMutate.
// * coprocessor calls (see ex. BulkDeleteEndpoint).
// So nonces are not really ever used by HBase. They could be by coprocs, and checkAnd...
return batchMutate(new MutationBatch(mutations, nonceGroup, nonce));
}
public OperationStatus[] batchMutate(Mutation[] mutations) throws IOException { public OperationStatus[] batchMutate(Mutation[] mutations) throws IOException {
return batchMutate(mutations, false); return batchMutate(mutations, HConstants.NO_NONCE, HConstants.NO_NONCE);
}
/**
* Replay a batch of mutations.
* @param mutations mutations to replay.
* @return
* @throws IOException
*/
public OperationStatus[] batchReplay(HLogSplitter.MutationReplay[] mutations)
throws IOException {
return batchMutate(new ReplayBatch(mutations));
} }
/** /**
@ -1935,21 +2031,16 @@ public class HRegion implements HeapSize { // , Writable{
* OperationStatusCode and the exceptionMessage if any. * OperationStatusCode and the exceptionMessage if any.
* @throws IOException * @throws IOException
*/ */
OperationStatus[] batchMutate(Mutation[] mutations, boolean isReplay) OperationStatus[] batchMutate(BatchOperationInProgress<?> batchOp) throws IOException {
throws IOException {
BatchOperationInProgress<Mutation> batchOp =
new BatchOperationInProgress<Mutation>(mutations);
boolean initialized = false; boolean initialized = false;
while (!batchOp.isDone()) { while (!batchOp.isDone()) {
if (!isReplay) { if (!batchOp.isInReplay()) {
checkReadOnly(); checkReadOnly();
} }
checkResources(); checkResources();
long newSize; long newSize;
if (isReplay) { if (batchOp.isInReplay()) {
startRegionOperation(Operation.REPLAY_BATCH_MUTATE); startRegionOperation(Operation.REPLAY_BATCH_MUTATE);
} else { } else {
startRegionOperation(Operation.BATCH_MUTATE); startRegionOperation(Operation.BATCH_MUTATE);
@ -1957,13 +2048,13 @@ public class HRegion implements HeapSize { // , Writable{
try { try {
if (!initialized) { if (!initialized) {
if (!isReplay) { if (!batchOp.isInReplay()) {
this.writeRequestsCount.increment(); this.writeRequestsCount.increment();
doPreMutationHook(batchOp); doPreMutationHook(batchOp);
} }
initialized = true; initialized = true;
} }
long addedSize = doMiniBatchMutation(batchOp, isReplay); long addedSize = doMiniBatchMutation(batchOp);
newSize = this.addAndGetGlobalMemstoreSize(addedSize); newSize = this.addAndGetGlobalMemstoreSize(addedSize);
} finally { } finally {
closeRegionOperation(); closeRegionOperation();
@ -1976,13 +2067,13 @@ public class HRegion implements HeapSize { // , Writable{
} }
private void doPreMutationHook(BatchOperationInProgress<Mutation> batchOp) private void doPreMutationHook(BatchOperationInProgress<?> batchOp)
throws IOException { throws IOException {
/* Run coprocessor pre hook outside of locks to avoid deadlock */ /* Run coprocessor pre hook outside of locks to avoid deadlock */
WALEdit walEdit = new WALEdit(); WALEdit walEdit = new WALEdit();
if (coprocessorHost != null) { if (coprocessorHost != null) {
for (int i = 0 ; i < batchOp.operations.length; i++) { for (int i = 0 ; i < batchOp.operations.length; i++) {
Mutation m = batchOp.operations[i]; Mutation m = batchOp.getMutation(i);
if (m instanceof Put) { if (m instanceof Put) {
if (coprocessorHost.prePut((Put) m, walEdit, m.getDurability())) { if (coprocessorHost.prePut((Put) m, walEdit, m.getDurability())) {
// pre hook says skip this Put // pre hook says skip this Put
@ -2011,9 +2102,8 @@ public class HRegion implements HeapSize { // , Writable{
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
private long doMiniBatchMutation(BatchOperationInProgress<Mutation> batchOp, private long doMiniBatchMutation(BatchOperationInProgress<?> batchOp) throws IOException {
boolean isInReplay) throws IOException { boolean isInReplay = batchOp.isInReplay();
// variable to note if all Put items are for the same CF -- metrics related // variable to note if all Put items are for the same CF -- metrics related
boolean putsCfSetConsistent = true; boolean putsCfSetConsistent = true;
//The set of columnFamilies first seen for Put. //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. //The set of columnFamilies first seen for Delete.
Set<byte[]> deletesCfSet = null; Set<byte[]> deletesCfSet = null;
long currentNonceGroup = HConstants.NO_NONCE, currentNonce = HConstants.NO_NONCE;
WALEdit walEdit = new WALEdit(isInReplay); WALEdit walEdit = new WALEdit(isInReplay);
MultiVersionConsistencyControl.WriteEntry w = null; MultiVersionConsistencyControl.WriteEntry w = null;
long txid = 0; long txid = 0;
@ -2046,7 +2137,7 @@ public class HRegion implements HeapSize { // , Writable{
int numReadyToWrite = 0; int numReadyToWrite = 0;
long now = EnvironmentEdgeManager.currentTimeMillis(); long now = EnvironmentEdgeManager.currentTimeMillis();
while (lastIndexExclusive < batchOp.operations.length) { while (lastIndexExclusive < batchOp.operations.length) {
Mutation mutation = batchOp.operations[lastIndexExclusive]; Mutation mutation = batchOp.getMutation(lastIndexExclusive);
boolean isPutMutation = mutation instanceof Put; boolean isPutMutation = mutation instanceof Put;
Map<byte[], List<Cell>> familyMap = mutation.getFamilyCellMap(); Map<byte[], List<Cell>> familyMap = mutation.getFamilyCellMap();
@ -2145,7 +2236,7 @@ public class HRegion implements HeapSize { // , Writable{
if (batchOp.retCodeDetails[i].getOperationStatusCode() if (batchOp.retCodeDetails[i].getOperationStatusCode()
!= OperationStatusCode.NOT_RUN) continue; != OperationStatusCode.NOT_RUN) continue;
Mutation mutation = batchOp.operations[i]; Mutation mutation = batchOp.getMutation(i);
if (mutation instanceof Put) { if (mutation instanceof Put) {
updateKVTimestamps(familyMaps[i].values(), byteNow); updateKVTimestamps(familyMaps[i].values(), byteNow);
noOfPuts++; noOfPuts++;
@ -2167,7 +2258,7 @@ public class HRegion implements HeapSize { // , Writable{
// calling the pre CP hook for batch mutation // calling the pre CP hook for batch mutation
if (!isInReplay && coprocessorHost != null) { if (!isInReplay && coprocessorHost != null) {
MiniBatchOperationInProgress<Mutation> miniBatchOp = MiniBatchOperationInProgress<Mutation> miniBatchOp =
new MiniBatchOperationInProgress<Mutation>(batchOp.operations, new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive); batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
if (coprocessorHost.preBatchMutate(miniBatchOp)) return 0L; if (coprocessorHost.preBatchMutate(miniBatchOp)) return 0L;
} }
@ -2193,6 +2284,7 @@ public class HRegion implements HeapSize { // , Writable{
// ------------------------------------ // ------------------------------------
// STEP 4. Build WAL edit // STEP 4. Build WAL edit
// ---------------------------------- // ----------------------------------
boolean hasWalAppends = false;
Durability durability = Durability.USE_DEFAULT; Durability durability = Durability.USE_DEFAULT;
for (int i = firstIndex; i < lastIndexExclusive; i++) { for (int i = firstIndex; i < lastIndexExclusive; i++) {
// Skip puts that were determined to be invalid during preprocessing // 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; batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
Mutation m = batchOp.operations[i]; Mutation m = batchOp.getMutation(i);
Durability tmpDur = getEffectiveDurability(m.getDurability()); Durability tmpDur = getEffectiveDurability(m.getDurability());
if (tmpDur.ordinal() > durability.ordinal()) { if (tmpDur.ordinal() > durability.ordinal()) {
durability = tmpDur; durability = tmpDur;
@ -2212,6 +2304,22 @@ public class HRegion implements HeapSize { // , Writable{
continue; 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 // Add WAL edits by CP
WALEdit fromCP = batchOp.walEditsFromCoprocessors[i]; WALEdit fromCP = batchOp.walEditsFromCoprocessors[i];
if (fromCP != null) { 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) { if (walEdit.size() > 0) {
txid = this.log.appendNoSync(this.getRegionInfo(), this.htableDescriptor.getTableName(), 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. // STEP 7. Sync wal.
// ------------------------- // -------------------------
if (walEdit.size() > 0) { if (hasWalAppends) {
syncOrDefer(txid, durability); syncOrDefer(txid, durability);
} }
walSyncSuccessful = true; walSyncSuccessful = true;
// calling the post CP hook for batch mutation // calling the post CP hook for batch mutation
if (!isInReplay && coprocessorHost != null) { if (!isInReplay && coprocessorHost != null) {
MiniBatchOperationInProgress<Mutation> miniBatchOp = MiniBatchOperationInProgress<Mutation> miniBatchOp =
new MiniBatchOperationInProgress<Mutation>(batchOp.operations, new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive); batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
coprocessorHost.postBatchMutate(miniBatchOp); coprocessorHost.postBatchMutate(miniBatchOp);
} }
@ -2274,7 +2384,7 @@ public class HRegion implements HeapSize { // , Writable{
!= OperationStatusCode.SUCCESS) { != OperationStatusCode.SUCCESS) {
continue; continue;
} }
Mutation m = batchOp.operations[i]; Mutation m = batchOp.getMutation(i);
if (m instanceof Put) { if (m instanceof Put) {
coprocessorHost.postPut((Put) m, walEdit, m.getDurability()); coprocessorHost.postPut((Put) m, walEdit, m.getDurability());
} else { } else {
@ -2362,10 +2472,10 @@ public class HRegion implements HeapSize { // , Writable{
boolean isPut = w instanceof Put; boolean isPut = w instanceof Put;
if (!isPut && !(w instanceof Delete)) if (!isPut && !(w instanceof Delete))
throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action must " + throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action must " +
"be Put or Delete"); "be Put or Delete");
if (!Bytes.equals(row, w.getRow())) { if (!Bytes.equals(row, w.getRow())) {
throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action's " + throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action's " +
"getRow must match the passed row"); "getRow must match the passed row");
} }
startRegionOperation(); startRegionOperation();
@ -2435,9 +2545,10 @@ public class HRegion implements HeapSize { // , Writable{
} }
} }
private void doBatchMutate(Mutation mutation) throws IOException, private void doBatchMutate(Mutation mutation) throws IOException, DoNotRetryIOException {
org.apache.hadoop.hbase.DoNotRetryIOException { // Currently this is only called for puts and deletes, so no nonces.
OperationStatus[] batchMutate = this.batchMutate(new Mutation[] { mutation }); OperationStatus[] batchMutate = this.batchMutate(new Mutation[] { mutation },
HConstants.NO_NONCE, HConstants.NO_NONCE);
if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) { if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) {
throw new FailedSanityCheckException(batchMutate[0].getExceptionMsg()); throw new FailedSanityCheckException(batchMutate[0].getExceptionMsg());
} else if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.BAD_FAMILY)) { } 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 * called when a Put/Delete has updated memstore but subequently fails to update
* the wal. This method is then invoked to rollback the memstore. * 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, Map<byte[], List<Cell>>[] familyMaps,
int start, int end) { int start, int end) {
int kvsRolledback = 0; int kvsRolledback = 0;
@ -2901,6 +3012,7 @@ public class HRegion implements HeapSize { // , Writable{
HLog.Entry entry; HLog.Entry entry;
Store store = null; Store store = null;
boolean reported_once = false; boolean reported_once = false;
ServerNonceManager ng = this.rsServices == null ? null : this.rsServices.getNonceManager();
try { try {
// How many edits seen before we check elapsed time // How many edits seen before we check elapsed time
@ -2916,6 +3028,10 @@ public class HRegion implements HeapSize { // , Writable{
HLogKey key = entry.getKey(); HLogKey key = entry.getKey();
WALEdit val = entry.getEdit(); WALEdit val = entry.getEdit();
if (ng != null) { // some test, or nonces disabled
ng.reportOperationFromWal(key.getNonceGroup(), key.getNonce(), key.getWriteTime());
}
if (reporter != null) { if (reporter != null) {
intervalEdits += val.size(); intervalEdits += val.size();
if (intervalEdits >= interval) { if (intervalEdits >= interval) {
@ -4395,35 +4511,47 @@ public class HRegion implements HeapSize { // , Writable{
} }
public void mutateRow(RowMutations rm) throws IOException { 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())); 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. * Perform atomic mutations within the region.
* @param mutations The list of mutations to perform. * @param mutations The list of mutations to perform.
* <code>mutations</code> can contain operations for multiple rows. * <code>mutations</code> can contain operations for multiple rows.
* Caller has to ensure that all rows are contained in this region. * Caller has to ensure that all rows are contained in this region.
* @param rowsToLock Rows to lock * @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 * If multiple rows are locked care should be taken that
* <code>rowsToLock</code> is sorted in order to avoid deadlocks. * <code>rowsToLock</code> is sorted in order to avoid deadlocks.
* @throws IOException * @throws IOException
*/ */
public void mutateRowsWithLocks(Collection<Mutation> mutations, public void mutateRowsWithLocks(Collection<Mutation> mutations,
Collection<byte[]> rowsToLock) throws IOException { Collection<byte[]> rowsToLock, long nonceGroup, long nonce) throws IOException {
MultiRowMutationProcessor proc = new MultiRowMutationProcessor(mutations, rowsToLock);
MultiRowMutationProcessor proc = processRowsWithLocks(proc, -1, nonceGroup, nonce);
new MultiRowMutationProcessor(mutations, rowsToLock);
processRowsWithLocks(proc, -1);
} }
/** /**
* Performs atomic multiple reads and writes on a given row. * Performs atomic multiple reads and writes on a given row.
* *
* @param processor The object defines the reads and writes to a 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 { 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 processor The object defines the reads and writes to a row.
* @param timeout The timeout of the processor.process() execution * @param timeout The timeout of the processor.process() execution
* Use a negative number to switch off the time bound * 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) public void processRowsWithLocks(RowProcessor<?,?> processor, long timeout,
throws IOException { long nonceGroup, long nonce) throws IOException {
for (byte[] row : processor.getRowsToLock()) { for (byte[] row : processor.getRowsToLock()) {
checkRow(row, "processRowsWithLocks"); checkRow(row, "processRowsWithLocks");
@ -4506,7 +4636,7 @@ public class HRegion implements HeapSize { // , Writable{
if (!walEdit.isEmpty()) { if (!walEdit.isEmpty()) {
txid = this.log.appendNoSync(this.getRegionInfo(), txid = this.log.appendNoSync(this.getRegionInfo(),
this.htableDescriptor.getTableName(), walEdit, processor.getClusterIds(), now, this.htableDescriptor.getTableName(), walEdit, processor.getClusterIds(), now,
this.htableDescriptor, this.sequenceId); this.htableDescriptor, this.sequenceId, true, nonceGroup, nonce);
} }
// 8. Release region lock // 8. Release region lock
if (locked) { 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 // TODO: There's a lot of boiler plate code identical
// to increment... See how to better unify that. // to increment... See how to better unify that.
/** /**
@ -4618,7 +4752,7 @@ public class HRegion implements HeapSize { // , Writable{
* @return new keyvalues after increment * @return new keyvalues after increment
* @throws IOException * @throws IOException
*/ */
public Result append(Append append) public Result append(Append append, long nonceGroup, long nonce)
throws IOException { throws IOException {
byte[] row = append.getRow(); byte[] row = append.getRow();
checkRow(row, "append"); checkRow(row, "append");
@ -4749,7 +4883,8 @@ public class HRegion implements HeapSize { // , Writable{
// as a Put. // as a Put.
txid = this.log.appendNoSync(this.getRegionInfo(), txid = this.log.appendNoSync(this.getRegionInfo(),
this.htableDescriptor.getTableName(), walEdits, new ArrayList<UUID>(), this.htableDescriptor.getTableName(), walEdits, new ArrayList<UUID>(),
EnvironmentEdgeManager.currentTimeMillis(), this.htableDescriptor, this.sequenceId); EnvironmentEdgeManager.currentTimeMillis(), this.htableDescriptor, this.sequenceId,
true, nonceGroup, nonce);
} else { } else {
recordMutationWithoutWal(append.getFamilyCellMap()); recordMutationWithoutWal(append.getFamilyCellMap());
} }
@ -4801,13 +4936,17 @@ public class HRegion implements HeapSize { // , Writable{
return append.isReturnResults() ? Result.create(allKVs) : null; 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. * Perform one or more increment operations on a row.
* @param increment * @param increment
* @return new keyvalues after increment * @return new keyvalues after increment
* @throws IOException * @throws IOException
*/ */
public Result increment(Increment increment) public Result increment(Increment increment, long nonceGroup, long nonce)
throws IOException { throws IOException {
byte [] row = increment.getRow(); byte [] row = increment.getRow();
checkRow(row, "increment"); checkRow(row, "increment");
@ -4923,7 +5062,8 @@ public class HRegion implements HeapSize { // , Writable{
// as a Put. // as a Put.
txid = this.log.appendNoSync(this.getRegionInfo(), txid = this.log.appendNoSync(this.getRegionInfo(),
this.htableDescriptor.getTableName(), walEdits, new ArrayList<UUID>(), this.htableDescriptor.getTableName(), walEdits, new ArrayList<UUID>(),
EnvironmentEdgeManager.currentTimeMillis(), this.htableDescriptor, this.sequenceId); EnvironmentEdgeManager.currentTimeMillis(), this.htableDescriptor, this.sequenceId,
true, nonceGroup, nonce);
} else { } else {
recordMutationWithoutWal(increment.getFamilyCellMap()); recordMutationWithoutWal(increment.getFamilyCellMap());
} }

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.regionserver; package org.apache.hadoop.hbase.regionserver;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.Thread.UncaughtExceptionHandler; import java.lang.Thread.UncaughtExceptionHandler;
import java.lang.annotation.Retention; import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy; 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.client.Scan;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException; 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.OutOfOrderScannerNextException;
import org.apache.hadoop.hbase.exceptions.RegionMovedException; import org.apache.hadoop.hbase.exceptions.RegionMovedException;
import org.apache.hadoop.hbase.exceptions.RegionOpeningException; import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
@ -441,6 +443,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
/** The health check chore. */ /** The health check chore. */
private HealthCheckChore healthCheckChore; private HealthCheckChore healthCheckChore;
/** The nonce manager chore. */
private Chore nonceManagerChore;
/** /**
* The server name the Master sees us as. Its made from the hostname the * 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 * 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 // Table level lock manager for locking for region operations
private TableLockManager tableLockManager; 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; private UserProvider userProvider;
/** /**
@ -529,6 +554,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
this.sleeper = new Sleeper(this.msgInterval, this); 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( this.maxScannerResultSize = conf.getLong(
HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY, HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE); 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 // Create the thread to clean the moved regions list
movedRegionsCleaner = MovedRegionsCleaner.createAndStart(this); 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 // Setup RPC client for master communication
rpcClient = new RpcClient(conf, clusterId, new InetSocketAddress( rpcClient = new RpcClient(conf, clusterId, new InetSocketAddress(
this.isa.getAddress(), 0)); this.isa.getAddress(), 0));
@ -910,6 +943,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
if (this.healthCheckChore != null) { if (this.healthCheckChore != null) {
this.healthCheckChore.interrupt(); this.healthCheckChore.interrupt();
} }
if (this.nonceManagerChore != null) {
this.nonceManagerChore.interrupt();
}
// Stop the snapshot handler, forcefully killing all running tasks // Stop the snapshot handler, forcefully killing all running tasks
try { try {
@ -1556,8 +1592,11 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
Threads.setDaemonThreadRunning(this.periodicFlusher.getThread(), n + Threads.setDaemonThreadRunning(this.periodicFlusher.getThread(), n +
".periodicFlusher", uncaughtExceptionHandler); ".periodicFlusher", uncaughtExceptionHandler);
if (this.healthCheckChore != null) { if (this.healthCheckChore != null) {
Threads Threads.setDaemonThreadRunning(this.healthCheckChore.getThread(), n + ".healthChecker",
.setDaemonThreadRunning(this.healthCheckChore.getThread(), n + ".healthChecker", uncaughtExceptionHandler);
}
if (this.nonceManagerChore != null) {
Threads.setDaemonThreadRunning(this.nonceManagerChore.getThread(), n + ".nonceCleaner",
uncaughtExceptionHandler); uncaughtExceptionHandler);
} }
@ -1811,6 +1850,9 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
* have already been called. * have already been called.
*/ */
protected void join() { protected void join() {
if (this.nonceManagerChore != null) {
Threads.shutdown(this.nonceManagerChore.getThread());
}
Threads.shutdown(this.compactionChecker.getThread()); Threads.shutdown(this.compactionChecker.getThread());
Threads.shutdown(this.periodicFlusher.getThread()); Threads.shutdown(this.periodicFlusher.getThread());
this.cacheFlusher.join(); this.cacheFlusher.join();
@ -2826,15 +2868,19 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
if (!region.getRegionInfo().isMetaTable()) { if (!region.getRegionInfo().isMetaTable()) {
cacheFlusher.reclaimMemStoreMemory(); cacheFlusher.reclaimMemStoreMemory();
} }
long nonceGroup = request.hasNonceGroup()
? request.getNonceGroup() : HConstants.NO_NONCE;
Result r = null; Result r = null;
Boolean processed = null; Boolean processed = null;
MutationType type = mutation.getMutateType(); MutationType type = mutation.getMutateType();
switch (type) { switch (type) {
case APPEND: case APPEND:
r = append(region, mutation, cellScanner); // TODO: this doesn't actually check anything.
r = append(region, mutation, cellScanner, nonceGroup);
break; break;
case INCREMENT: case INCREMENT:
r = increment(region, mutation, cellScanner); // TODO: this doesn't actually check anything.
r = increment(region, mutation, cellScanner, nonceGroup);
break; break;
case PUT: case PUT:
Put put = ProtobufUtil.toPut(mutation, cellScanner); Put put = ProtobufUtil.toPut(mutation, cellScanner);
@ -3251,6 +3297,8 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
CellScanner cellScanner = controller != null ? controller.cellScanner(): null; CellScanner cellScanner = controller != null ? controller.cellScanner(): null;
if (controller != null) controller.setCellScanner(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. // this will contain all the cells that we need to return. It's created later, if needed.
List<CellScannable> cellsToReturn = null; List<CellScannable> cellsToReturn = null;
MultiResponse.Builder responseBuilder = MultiResponse.newBuilder(); MultiResponse.Builder responseBuilder = MultiResponse.newBuilder();
@ -3279,7 +3327,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
} else { } else {
// doNonAtomicRegionMutation manages the exception internally // doNonAtomicRegionMutation manages the exception internally
cellsToReturn = doNonAtomicRegionMutation(region, regionAction, cellScanner, cellsToReturn = doNonAtomicRegionMutation(region, regionAction, cellScanner,
regionActionResultBuilder, cellsToReturn); regionActionResultBuilder, cellsToReturn, nonceGroup);
} }
responseBuilder.addRegionActionResult(regionActionResultBuilder.build()); responseBuilder.addRegionActionResult(regionActionResultBuilder.build());
} }
@ -3303,7 +3351,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
*/ */
private List<CellScannable> doNonAtomicRegionMutation(final HRegion region, private List<CellScannable> doNonAtomicRegionMutation(final HRegion region,
final RegionAction actions, final CellScanner cellScanner, 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 // 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 // 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 // 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) { switch (type) {
case APPEND: case APPEND:
r = append(region, action.getMutation(), cellScanner); r = append(region, action.getMutation(), cellScanner, nonceGroup);
break; break;
case INCREMENT: case INCREMENT:
r = increment(region, action.getMutation(), cellScanner); r = increment(region, action.getMutation(), cellScanner, nonceGroup);
break; break;
case PUT: case PUT:
case DELETE: case DELETE:
@ -3862,12 +3910,18 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
entries.get(0).getKey().getEncodedRegionName().toStringUtf8()); entries.get(0).getKey().getEncodedRegionName().toStringUtf8());
RegionCoprocessorHost coprocessorHost = region.getCoprocessorHost(); RegionCoprocessorHost coprocessorHost = region.getCoprocessorHost();
List<Pair<HLogKey, WALEdit>> walEntries = new ArrayList<Pair<HLogKey, WALEdit>>(); 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) { 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 : Pair<HLogKey, WALEdit> walEntry = (coprocessorHost == null) ? null :
new Pair<HLogKey, WALEdit>(); new Pair<HLogKey, WALEdit>();
List<Pair<MutationType, Mutation>> edits = HLogSplitter.getMutationsFromWALEntry(entry, List<HLogSplitter.MutationReplay> edits =
cells, walEntry); HLogSplitter.getMutationsFromWALEntry(entry, cells, walEntry);
if (coprocessorHost != null) { if (coprocessorHost != null) {
// Start coprocessor replay here. The coprocessor is for each WALEdit instead of a // Start coprocessor replay here. The coprocessor is for each WALEdit instead of a
// KeyValue. // KeyValue.
@ -3882,7 +3936,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
} }
if (!mutations.isEmpty()) { if (!mutations.isEmpty()) {
OperationStatus[] result = doBatchOp(region, mutations, true); OperationStatus[] result = doReplayBatchOp(region, mutations);
// check if it's a partial success // check if it's a partial success
for (int i = 0; result != null && i < result.length; i++) { for (int i = 0; result != null && i < result.length; i++) {
if (result[i] != OperationStatus.SUCCESS) { if (result[i] != OperationStatus.SUCCESS) {
@ -3987,7 +4041,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
* @throws IOException * @throws IOException
*/ */
protected Result append(final HRegion region, 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(); long before = EnvironmentEdgeManager.currentTimeMillis();
Append append = ProtobufUtil.toAppend(m, cellScanner); Append append = ProtobufUtil.toAppend(m, cellScanner);
Result r = null; Result r = null;
@ -3995,7 +4049,14 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
r = region.getCoprocessorHost().preAppend(append); r = region.getCoprocessorHost().preAppend(append);
} }
if (r == null) { 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) { if (region.getCoprocessorHost() != null) {
region.getCoprocessorHost().postAppend(append, r); region.getCoprocessorHost().postAppend(append, r);
} }
@ -4013,8 +4074,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
* @throws IOException * @throws IOException
*/ */
protected Result increment(final HRegion region, final MutationProto mutation, protected Result increment(final HRegion region, final MutationProto mutation,
final CellScanner cells) final CellScanner cells, long nonceGroup) throws IOException {
throws IOException {
long before = EnvironmentEdgeManager.currentTimeMillis(); long before = EnvironmentEdgeManager.currentTimeMillis();
Increment increment = ProtobufUtil.toIncrement(mutation, cells); Increment increment = ProtobufUtil.toIncrement(mutation, cells);
Result r = null; Result r = null;
@ -4022,7 +4082,14 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
r = region.getCoprocessorHost().preIncrement(increment); r = region.getCoprocessorHost().preIncrement(increment);
} }
if (r == null) { 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) { if (region.getCoprocessorHost() != null) {
r = region.getCoprocessorHost().postIncrement(increment, r); r = region.getCoprocessorHost().postIncrement(increment, r);
} }
@ -4031,6 +4098,49 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
return r; 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. * Execute a list of Put/Delete mutations.
* *
@ -4063,7 +4173,7 @@ public class HRegionServer implements ClientProtos.ClientService.BlockingInterfa
cacheFlusher.reclaimMemStoreMemory(); cacheFlusher.reclaimMemStoreMemory();
} }
OperationStatus codes[] = region.batchMutate(mArray, false); OperationStatus codes[] = region.batchMutate(mArray);
for (i = 0; i < codes.length; i++) { for (i = 0; i < codes.length; i++) {
int index = mutations.get(i).getIndex(); int index = mutations.get(i).getIndex();
Exception e = null; 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. * constructing MultiResponse to save a possible loop if caller doesn't need MultiResponse.
* @param region * @param region
* @param mutations * @param mutations
* @param isReplay
* @return an array of OperationStatus which internally contains the OperationStatusCode and the * @return an array of OperationStatus which internally contains the OperationStatusCode and the
* exceptionMessage if any * exceptionMessage if any
* @throws IOException * @throws IOException
*/ */
protected OperationStatus [] doBatchOp(final HRegion region, protected OperationStatus [] doReplayBatchOp(final HRegion region,
final List<Pair<MutationType, Mutation>> mutations, boolean isReplay) final List<HLogSplitter.MutationReplay> mutations) throws IOException {
throws IOException { HLogSplitter.MutationReplay[] mArray = new HLogSplitter.MutationReplay[mutations.size()];
Mutation[] mArray = new Mutation[mutations.size()];
long before = EnvironmentEdgeManager.currentTimeMillis(); long before = EnvironmentEdgeManager.currentTimeMillis();
boolean batchContainsPuts = false, batchContainsDelete = false; boolean batchContainsPuts = false, batchContainsDelete = false;
try { try {
int i = 0; int i = 0;
for (Pair<MutationType, Mutation> m : mutations) { for (HLogSplitter.MutationReplay m : mutations) {
if (m.getFirst() == MutationType.PUT) { if (m.type == MutationType.PUT) {
batchContainsPuts = true; batchContainsPuts = true;
} else { } else {
batchContainsDelete = true; batchContainsDelete = true;
} }
mArray[i++] = m.getSecond(); mArray[i++] = m;
} }
requestCount.add(mutations.size()); requestCount.add(mutations.size());
if (!region.getRegionInfo().isMetaTable()) { if (!region.getRegionInfo().isMetaTable()) {
cacheFlusher.reclaimMemStoreMemory(); cacheFlusher.reclaimMemStoreMemory();
} }
return region.batchMutate(mArray, isReplay); return region.batchReplay(mArray);
} finally { } finally {
long after = EnvironmentEdgeManager.currentTimeMillis(); long after = EnvironmentEdgeManager.currentTimeMillis();
if (batchContainsPuts) { if (batchContainsPuts) {

View File

@ -115,4 +115,10 @@ public interface RegionServerServices
* @return set of recovering regions on the hosting region server * @return set of recovering regions on the hosting region server
*/ */
Map<String, HRegion> getRecoveringRegions(); 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.Trace;
import org.cloudera.htrace.TraceScope; 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 * HLog stores all the edits to the HStore. Its the hbase write-ahead-log
* implementation. * implementation.
@ -898,21 +900,16 @@ class FSHLog implements HLog, Syncable {
* @return New log key. * @return New log key.
*/ */
protected HLogKey makeKey(byte[] encodedRegionName, TableName tableName, long seqnum, protected HLogKey makeKey(byte[] encodedRegionName, TableName tableName, long seqnum,
long now, List<UUID> clusterIds) { long now, List<UUID> clusterIds, long nonceGroup, long nonce) {
return new HLogKey(encodedRegionName, tableName, seqnum, now, clusterIds); return new HLogKey(encodedRegionName, tableName, seqnum, now, clusterIds, nonceGroup, nonce);
} }
@Override @Override
@VisibleForTesting
public void append(HRegionInfo info, TableName tableName, WALEdit edits, public void append(HRegionInfo info, TableName tableName, WALEdit edits,
final long now, HTableDescriptor htd, AtomicLong sequenceId) final long now, HTableDescriptor htd, AtomicLong sequenceId) throws IOException {
throws IOException { append(info, tableName, edits, new ArrayList<UUID>(), now, htd, true, true, sequenceId,
append(info, tableName, edits, now, htd, true, sequenceId); HConstants.NO_NONCE, HConstants.NO_NONCE);
}
@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);
} }
/** /**
@ -944,8 +941,8 @@ class FSHLog implements HLog, Syncable {
*/ */
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")
private long append(HRegionInfo info, TableName tableName, WALEdit edits, List<UUID> clusterIds, private long append(HRegionInfo info, TableName tableName, WALEdit edits, List<UUID> clusterIds,
final long now, HTableDescriptor htd, boolean doSync, boolean isInMemstore, AtomicLong sequenceId) final long now, HTableDescriptor htd, boolean doSync, boolean isInMemstore,
throws IOException { AtomicLong sequenceId, long nonceGroup, long nonce) throws IOException {
if (edits.isEmpty()) return this.unflushedEntries.get(); if (edits.isEmpty()) return this.unflushedEntries.get();
if (this.closed) { if (this.closed) {
throw new IOException("Cannot append; log is closed"); throw new IOException("Cannot append; log is closed");
@ -966,7 +963,8 @@ class FSHLog implements HLog, Syncable {
// actual name. // actual name.
byte [] encodedRegionName = info.getEncodedNameAsBytes(); byte [] encodedRegionName = info.getEncodedNameAsBytes();
if (isInMemstore) this.oldestUnflushedSeqNums.putIfAbsent(encodedRegionName, seqNum); 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); doWrite(info, logKey, edits, htd);
this.numEntries.incrementAndGet(); this.numEntries.incrementAndGet();
txid = this.unflushedEntries.incrementAndGet(); txid = this.unflushedEntries.incrementAndGet();
@ -975,6 +973,8 @@ class FSHLog implements HLog, Syncable {
} }
this.latestSequenceNums.put(encodedRegionName, seqNum); 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 // Sync if catalog region, and if not then check if that table supports
// deferred log flushing // deferred log flushing
if (doSync && if (doSync &&
@ -991,9 +991,10 @@ class FSHLog implements HLog, Syncable {
@Override @Override
public long appendNoSync(HRegionInfo info, TableName tableName, WALEdit edits, public long appendNoSync(HRegionInfo info, TableName tableName, WALEdit edits,
List<UUID> clusterIds, final long now, HTableDescriptor htd, AtomicLong sequenceId) List<UUID> clusterIds, final long now, HTableDescriptor htd, AtomicLong sequenceId,
throws IOException { boolean isInMemstore, long nonceGroup, long nonce) throws IOException {
return append(info, tableName, edits, clusterIds, now, htd, false, true, sequenceId); 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.hbase.protobuf.generated.WALProtos.WALTrailer;
import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.Writable;
import com.google.common.annotations.VisibleForTesting;
@InterfaceAudience.Private @InterfaceAudience.Private
// TODO: Rename interface to WAL // TODO: Rename interface to WAL
@ -260,24 +262,10 @@ public interface HLog {
* except it causes a sync on the log * except it causes a sync on the log
* @param sequenceId of the region. * @param sequenceId of the region.
*/ */
@VisibleForTesting
public void append(HRegionInfo info, TableName tableName, WALEdit edits, public void append(HRegionInfo info, TableName tableName, WALEdit edits,
final long now, HTableDescriptor htd, AtomicLong sequenceId) throws IOException; 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 * 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. * 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 * @return txid of this transaction
* @throws IOException * @throws IOException
*/ */
public long appendNoSync(HRegionInfo info, TableName tableName, WALEdit edits, long appendNoSync(HRegionInfo info, TableName tableName, WALEdit edits,
List<UUID> clusterIds, final long now, HTableDescriptor htd, AtomicLong sequenceId) throws IOException; 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? // TODO: Do we need all these versions of sync?
void hsync() throws IOException; 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.WritableComparable;
import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.io.WritableUtils;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
import com.google.protobuf.ZeroCopyLiteralByteString; import com.google.protobuf.ZeroCopyLiteralByteString;
@ -119,18 +120,23 @@ public class HLogKey implements WritableComparable<HLogKey> {
private NavigableMap<byte[], Integer> scopes; private NavigableMap<byte[], Integer> scopes;
private long nonceGroup = HConstants.NO_NONCE;
private long nonce = HConstants.NO_NONCE;
private CompressionContext compressionContext; private CompressionContext compressionContext;
public HLogKey() { public HLogKey() {
init(null, null, 0L, HConstants.LATEST_TIMESTAMP, 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, public HLogKey(final byte[] encodedRegionName, final TableName tablename, long logSeqNum,
final long now, UUID clusterId) { final long now, UUID clusterId) {
List<UUID> clusterIds = new ArrayList<UUID>(); List<UUID> clusterIds = new ArrayList<UUID>();
clusterIds.add(clusterId); 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) * @param clusterIds the clusters that have consumed the change(used in Replication)
*/ */
public HLogKey(final byte [] encodedRegionName, final TableName tablename, public HLogKey(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) {
init(encodedRegionName, tablename, logSeqNum, now, clusterIds); init(encodedRegionName, tablename, logSeqNum, now, clusterIds, nonceGroup, nonce);
} }
protected void init(final byte [] encodedRegionName, final TableName tablename, 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.logSeqNum = logSeqNum;
this.writeTime = now; this.writeTime = now;
this.clusterIds = clusterIds; this.clusterIds = clusterIds;
this.encodedRegionName = encodedRegionName; this.encodedRegionName = encodedRegionName;
this.tablename = tablename; this.tablename = tablename;
this.nonceGroup = nonceGroup;
this.nonce = nonce;
} }
/** /**
@ -192,6 +200,16 @@ public class HLogKey implements WritableComparable<HLogKey> {
return scopes; 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) { public void setScopes(NavigableMap<byte[], Integer> scopes) {
this.scopes = scopes; this.scopes = scopes;
} }
@ -435,6 +453,12 @@ public class HLogKey implements WritableComparable<HLogKey> {
} }
builder.setLogSequenceNumber(this.logSeqNum); builder.setLogSequenceNumber(this.logSeqNum);
builder.setWriteTime(writeTime); 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(); HBaseProtos.UUID.Builder uuidBuilder = HBaseProtos.UUID.newBuilder();
for (UUID clusterId : clusterIds) { for (UUID clusterId : clusterIds) {
uuidBuilder.setLeastSigBits(clusterId.getLeastSignificantBits()); uuidBuilder.setLeastSigBits(clusterId.getLeastSignificantBits());
@ -474,6 +498,12 @@ public class HLogKey implements WritableComparable<HLogKey> {
for (HBaseProtos.UUID clusterId : walKey.getClusterIdsList()) { for (HBaseProtos.UUID clusterId : walKey.getClusterIdsList()) {
clusterIds.add(new UUID(clusterId.getMostSigBits(), clusterId.getLeastSigBits())); 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; this.scopes = null;
if (walKey.getScopesCount() > 0) { if (walKey.getScopesCount() > 0) {
this.scopes = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR); 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 & * This function is used to construct mutations from a WALEntry. It also reconstructs HLogKey &
* WALEdit from the passed in WALEntry * WALEdit from the passed in WALEntry
@ -1846,16 +1861,16 @@ public class HLogSplitter {
* @return list of Pair<MutationType, Mutation> to be replayed * @return list of Pair<MutationType, Mutation> to be replayed
* @throws IOException * @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 { CellScanner cells, Pair<HLogKey, WALEdit> logEntry) throws IOException {
if (entry == null) { if (entry == null) {
// return an empty array // return an empty array
return new ArrayList<Pair<MutationType, Mutation>>(); return new ArrayList<MutationReplay>();
} }
int count = entry.getAssociatedCellCount(); int count = entry.getAssociatedCellCount();
List<Pair<MutationType, Mutation>> mutations = new ArrayList<Pair<MutationType, Mutation>>(); List<MutationReplay> mutations = new ArrayList<MutationReplay>();
Cell previousCell = null; Cell previousCell = null;
Mutation m = null; Mutation m = null;
HLogKey key = null; HLogKey key = null;
@ -1877,10 +1892,16 @@ public class HLogSplitter {
// Create new mutation // Create new mutation
if (CellUtil.isDelete(cell)) { if (CellUtil.isDelete(cell)) {
m = new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()); 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 { } else {
m = new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()); 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)) { if (CellUtil.isDelete(cell)) {
@ -1900,7 +1921,7 @@ public class HLogSplitter {
} }
key = new HLogKey(walKey.getEncodedRegionName().toByteArray(), TableName.valueOf(walKey key = new HLogKey(walKey.getEncodedRegionName().toByteArray(), TableName.valueOf(walKey
.getTableName().toByteArray()), walKey.getLogSequenceNumber(), walKey.getWriteTime(), .getTableName().toByteArray()), walKey.getLogSequenceNumber(), walKey.getWriteTime(),
clusterIds); clusterIds, walKey.getNonceGroup(), walKey.getNonce());
logEntry.setFirst(key); logEntry.setFirst(key);
logEntry.setSecond(val); logEntry.setSecond(val);
} }

View File

@ -20,8 +20,10 @@
package org.apache.hadoop.hbase.regionserver.wal; package org.apache.hadoop.hbase.regionserver.wal;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.NavigableSet; import java.util.NavigableSet;
import java.util.TreeSet; import java.util.TreeSet;
import java.util.UUID;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.regex.Matcher; import java.util.regex.Matcher;
import java.util.regex.Pattern; import java.util.regex.Pattern;
@ -262,8 +264,12 @@ public class HLogUtil {
public static void writeCompactionMarker(HLog log, HTableDescriptor htd, HRegionInfo info, public static void writeCompactionMarker(HLog log, HTableDescriptor htd, HRegionInfo info,
final CompactionDescriptor c, AtomicLong sequenceId) throws IOException { final CompactionDescriptor c, AtomicLong sequenceId) throws IOException {
WALEdit e = WALEdit.createCompaction(c); WALEdit e = WALEdit.createCompaction(c);
log.append(info, TableName.valueOf(c.getTableName().toByteArray()), e, long now = EnvironmentEdgeManager.currentTimeMillis();
EnvironmentEdgeManager.currentTimeMillis(), htd, false, sequenceId); 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()) { if (LOG.isTraceEnabled()) {
LOG.trace("Appended compaction marker " + TextFormat.shortDebugString(c)); LOG.trace("Appended compaction marker " + TextFormat.shortDebugString(c));
} }

View File

@ -171,8 +171,8 @@ class SnapshotLogSplitter implements Closeable {
} }
// Append Entry // Append Entry
key = new HLogKey(newRegionName, tableName, key = new HLogKey(newRegionName, tableName, key.getLogSeqNum(), key.getWriteTime(),
key.getLogSeqNum(), key.getWriteTime(), key.getClusterIds()); key.getClusterIds(), key.getNonceGroup(), key.getNonce());
writer.append(new HLog.Entry(key, entry.getEdit())); writer.append(new HLog.Entry(key, entry.getEdit()));
} }
} catch (IOException e) { } 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.Leases;
import org.apache.hadoop.hbase.regionserver.RegionServerAccounting; import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
import org.apache.hadoop.hbase.regionserver.RegionServerServices; 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.regionserver.wal.HLog;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -192,7 +193,6 @@ class MockRegionServerServices implements RegionServerServices {
@Override @Override
public HLog getWAL(HRegionInfo regionInfo) throws IOException { public HLog getWAL(HRegionInfo regionInfo) throws IOException {
// TODO Auto-generated method stub
return null; return null;
} }
@ -221,4 +221,10 @@ class MockRegionServerServices implements RegionServerServices {
public int getPriority(RPCProtos.RequestHeader header, Message param) { public int getPriority(RPCProtos.RequestHeader header, Message param) {
return 0; 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.lang.reflect.Field;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.ThreadPoolExecutor;
import org.apache.commons.logging.Log; 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.HBaseTestingUtility;
import org.apache.hadoop.hbase.MediumTests; import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.Waiter; 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.RpcClient;
import org.apache.hadoop.hbase.ipc.RpcServer; import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.JVMClusterUtil;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.log4j.Level; import org.apache.log4j.Level;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
@ -481,6 +484,96 @@ public class TestMultiParallel {
table.close(); 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) @Test(timeout=300000)
public void testBatchWithMixedActions() throws Exception { public void testBatchWithMixedActions() throws Exception {
LOG.info("test=testBatchWithMixedActions"); LOG.info("test=testBatchWithMixedActions");
@ -558,10 +651,13 @@ public class TestMultiParallel {
} }
private void validateResult(Object r1, byte[] qual, byte[] val) { private void validateResult(Object r1, byte[] qual, byte[] val) {
// TODO provide nice assert here or something.
Result r = (Result)r1; Result r = (Result)r1;
Assert.assertTrue(r.containsColumn(BYTES_FAMILY, qual)); 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() { 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.Leases;
import org.apache.hadoop.hbase.regionserver.RegionServerAccounting; import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
import org.apache.hadoop.hbase.regionserver.RegionServerServices; 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.regionserver.wal.HLog;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
@ -554,4 +555,9 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
UpdateFavoredNodesRequest request) throws ServiceException { UpdateFavoredNodesRequest request) throws ServiceException {
return null; 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_done;
import static org.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_err; 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.apache.hadoop.hbase.SplitLogCounters.tot_wkr_task_resigned;
import static org.junit.Assert.*;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
@ -35,6 +36,7 @@ import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator; import java.util.Iterator;
import java.util.LinkedList;
import java.util.List; import java.util.List;
import java.util.NavigableSet; import java.util.NavigableSet;
import java.util.Set; 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.SplitLogCounters;
import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.Delete; 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.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.Put;
import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException; 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.exceptions.RegionInRecoveryException;
import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch; import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
@ -172,7 +179,7 @@ public class TestDistributedLogSplitting {
try { try {
if (TEST_UTIL.getHBaseCluster() != null) { if (TEST_UTIL.getHBaseCluster() != null) {
for (MasterThread mt : TEST_UTIL.getHBaseCluster().getLiveMasterThreads()) { for (MasterThread mt : TEST_UTIL.getHBaseCluster().getLiveMasterThreads()) {
mt.getMaster().abort("closing...", new Exception("Trace info")); mt.getMaster().abort("closing...", null);
} }
} }
TEST_UTIL.shutdownMiniHBaseCluster(); TEST_UTIL.shutdownMiniHBaseCluster();
@ -273,6 +280,80 @@ public class TestDistributedLogSplitting {
zkw.close(); 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) @Test(timeout = 300000)
public void testLogReplayWithMetaRSDown() throws Exception { public void testLogReplayWithMetaRSDown() throws Exception {
LOG.info("testRecoveredEditsReplayWithMetaRSDown"); LOG.info("testRecoveredEditsReplayWithMetaRSDown");

View File

@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals;
import java.io.IOException; import java.io.IOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.SmallTests; import org.apache.hadoop.hbase.SmallTests;
import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
@ -220,7 +221,7 @@ public class TestProtobufUtil {
Increment increment = ProtobufUtil.toIncrement(proto, null); Increment increment = ProtobufUtil.toIncrement(proto, null);
assertEquals(mutateBuilder.build(), 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.assertNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import static org.mockito.Matchers.any; import static org.mockito.Matchers.*;
import static org.mockito.Matchers.anyLong;
import static org.mockito.Matchers.eq;
import static org.mockito.Mockito.never; import static org.mockito.Mockito.never;
import static org.mockito.Mockito.spy; import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times; import static org.mockito.Mockito.times;
@ -3800,10 +3798,11 @@ public class TestHRegion {
put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1")); put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1"));
put.setDurability(mutationDurability); put.setDurability(mutationDurability);
region.put(put); region.put(put);
//verify append called or not //verify append called or not
verify(log, expectAppend ? times(1) : never()) verify(log, expectAppend ? times(1) : never())
.appendNoSync((HRegionInfo)any(), eq(tableName), (WALEdit)any(), (List<UUID>)any(), .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 // verify sync called or not
if (expectSync || expectSyncFromLogSyncer) { 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.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
@ -104,6 +105,8 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool
byte[] value = new byte[valueSize]; byte[] value = new byte[valueSize];
Random rand = new Random(Thread.currentThread().getId()); Random rand = new Random(Thread.currentThread().getId());
HLog hlog = region.getLog(); HLog hlog = region.getLog();
ArrayList<UUID> clusters = new ArrayList<UUID>();
long nonce = HConstants.NO_NONCE;
try { try {
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
@ -114,8 +117,8 @@ public final class HLogPerformanceEvaluation extends Configured implements Tool
addFamilyMapToWALEdit(put.getFamilyCellMap(), walEdit); addFamilyMapToWALEdit(put.getFamilyCellMap(), walEdit);
HRegionInfo hri = region.getRegionInfo(); HRegionInfo hri = region.getRegionInfo();
if (this.noSync) { if (this.noSync) {
hlog.appendNoSync(hri, hri.getTable(), walEdit, new ArrayList<UUID>(), now, htd, hlog.appendNoSync(hri, hri.getTable(), walEdit, clusters, now, htd,
region.getSequenceId()); region.getSequenceId(), true, nonce, nonce);
} else { } else {
hlog.append(hri, hri.getTable(), walEdit, now, htd, region.getSequenceId()); hlog.append(hri, hri.getTable(), walEdit, now, htd, region.getSequenceId());
} }