Reform java client (#83)

This commit is contained in:
birdstorm 2020-09-07 14:13:09 +08:00 committed by GitHub
parent 9a0b8681db
commit 41aeed2679
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
48 changed files with 3597 additions and 993 deletions

33
pom.xml
View File

@ -12,10 +12,10 @@
<properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
<protobuf.version>3.1.0</protobuf.version>
<protobuf.version>3.5.1</protobuf.version>
<log4j.version>1.2.17</log4j.version>
<slf4j.version>1.7.16</slf4j.version>
<grpc.version>1.17.0</grpc.version>
<grpc.version>1.24.0</grpc.version>
<powermock.version>1.6.6</powermock.version>
<jackson.version>2.10.0</jackson.version>
<trove4j.version>3.0.1</trove4j.version>
@ -28,9 +28,28 @@
<dependencies>
<dependency>
<groupId>log4j</groupId>
<artifactId>log4j</artifactId>
<version>${log4j.version}</version>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
<version>${slf4j.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-log4j12</artifactId>
<version>${slf4j.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>jul-to-slf4j</artifactId>
<version>${slf4j.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>jcl-over-slf4j</artifactId>
<version>${slf4j.version}</version>
<scope>provided</scope>
</dependency>
<!-- grpc dependencies -->
<dependency>
@ -154,7 +173,7 @@
<plugin>
<groupId>org.xolstice.maven.plugins</groupId>
<artifactId>protobuf-maven-plugin</artifactId>
<version>0.5.0</version>
<version>0.6.1</version>
<configuration>
<protocArtifact>com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier}</protocArtifact>
<protoSourceRoot>${proto.folder}</protoSourceRoot>
@ -162,7 +181,7 @@
<param>**/*.proto</param>
</includes>
<pluginId>grpc-java</pluginId>
<pluginArtifact>io.grpc:protoc-gen-grpc-java:1.4.0:exe:${os.detected.classifier}</pluginArtifact>
<pluginArtifact>io.grpc:protoc-gen-grpc-java:${grpc.version}:exe:${os.detected.classifier}</pluginArtifact>
</configuration>
<executions>
<execution>

View File

@ -14,11 +14,15 @@
# limitations under the License.
#
kvproto_hash=a4759dfe3753ce136d252578340bb2b33633ccfa
CURRENT_DIR=`pwd`
TIKV_CLIENT_HOME="$(cd "`dirname "$0"`"/..; pwd)"
cd $TIKV_CLIENT_HOME
raft_rs_hash=14f007b443935aef51cb161c5b368b54fc8ed176
kvproto_hash=2cf9a243b8d589f345de1dbaa9eeffec6afbdc06
tipb_hash=c0b8f1a8c8395c319049600dc0efd278f1e26a0d
raft_rs_hash=b9891b673573fad77ebcf9bbe0969cf945841926
tipb_hash=c4d518eb1d60c21f05b028b36729e64610346dac
if [ -d "kvproto" ]; then
cd kvproto; git fetch -p; git checkout ${kvproto_hash}; cd ..
@ -37,3 +41,5 @@ if [ -d "tipb" ]; then
else
git clone https://github.com/pingcap/tipb; cd tipb; git checkout ${tipb_hash}; cd ..
fi
cd $CURRENT_DIR

View File

@ -23,7 +23,8 @@ import io.grpc.stub.AbstractStub;
import io.grpc.stub.ClientCalls;
import io.grpc.stub.StreamObserver;
import java.util.function.Supplier;
import org.apache.log4j.Logger;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.tikv.common.operation.ErrorHandler;
import org.tikv.common.policy.RetryMaxMs.Builder;
import org.tikv.common.policy.RetryPolicy;
@ -34,16 +35,29 @@ import org.tikv.common.util.ChannelFactory;
public abstract class AbstractGRPCClient<
BlockingStubT extends AbstractStub<BlockingStubT>, StubT extends AbstractStub<StubT>>
implements AutoCloseable {
protected final Logger logger = Logger.getLogger(this.getClass());
protected final TiConfiguration conf;
protected final Logger logger = LoggerFactory.getLogger(this.getClass());
protected final ChannelFactory channelFactory;
protected TiConfiguration conf;
protected BlockingStubT blockingStub;
protected StubT asyncStub;
protected AbstractGRPCClient(TiConfiguration conf, ChannelFactory channelFactory) {
this.conf = conf;
this.channelFactory = channelFactory;
}
protected TiConfiguration getConf() {
protected AbstractGRPCClient(
TiConfiguration conf,
ChannelFactory channelFactory,
BlockingStubT blockingStub,
StubT asyncStub) {
this.conf = conf;
this.channelFactory = channelFactory;
this.blockingStub = blockingStub;
this.asyncStub = asyncStub;
}
public TiConfiguration getConf() {
return conf;
}

View File

@ -25,19 +25,18 @@ import com.google.protobuf.ByteString;
import io.grpc.ManagedChannel;
import java.net.URI;
import java.util.List;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.*;
import java.util.function.Supplier;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.tikv.common.TiConfiguration.KVMode;
import org.tikv.common.codec.Codec.BytesCodec;
import org.tikv.common.codec.CodecDataOutput;
import org.tikv.common.exception.GrpcException;
import org.tikv.common.exception.TiClientInternalException;
import org.tikv.common.exception.TiKVException;
import org.tikv.common.meta.TiTimestamp;
import org.tikv.common.operation.PDErrorHandler;
import org.tikv.common.pd.PDUtils;
import org.tikv.common.region.TiRegion;
import org.tikv.common.util.BackOffer;
import org.tikv.common.util.ChannelFactory;
@ -51,12 +50,28 @@ import org.tikv.kvproto.Pdpb.*;
/** PDClient is thread-safe and suggested to be shared threads */
public class PDClient extends AbstractGRPCClient<PDBlockingStub, PDStub>
implements ReadOnlyPDClient {
private final Logger logger = LoggerFactory.getLogger(PDClient.class);
private RequestHeader header;
private TsoRequest tsoReq;
private volatile LeaderWrapper leaderWrapper;
private ScheduledExecutorService service;
private List<URI> pdAddrs;
private PDClient(TiConfiguration conf, ChannelFactory channelFactory) {
super(conf, channelFactory);
initCluster();
this.blockingStub = getBlockingStub();
this.asyncStub = getAsyncStub();
}
public static ReadOnlyPDClient create(TiConfiguration conf, ChannelFactory channelFactory) {
return createRaw(conf, channelFactory);
}
static PDClient createRaw(TiConfiguration conf, ChannelFactory channelFactory) {
return new PDClient(conf, channelFactory);
}
@Override
public TiTimestamp getTimestamp(BackOffer backOffer) {
Supplier<TsoRequest> request = () -> tsoReq;
@ -66,11 +81,35 @@ public class PDClient extends AbstractGRPCClient<PDBlockingStub, PDStub>
r -> r.getHeader().hasError() ? buildFromPdpbError(r.getHeader().getError()) : null,
this);
TsoResponse resp = callWithRetry(backOffer, PDGrpc.METHOD_TSO, request, handler);
TsoResponse resp = callWithRetry(backOffer, PDGrpc.getTsoMethod(), request, handler);
Timestamp timestamp = resp.getTimestamp();
return new TiTimestamp(timestamp.getPhysical(), timestamp.getLogical());
}
/**
* Sends request to pd to scatter region.
*
* @param region represents a region info
*/
void scatterRegion(TiRegion region, BackOffer backOffer) {
Supplier<ScatterRegionRequest> request =
() ->
ScatterRegionRequest.newBuilder().setHeader(header).setRegionId(region.getId()).build();
PDErrorHandler<ScatterRegionResponse> handler =
new PDErrorHandler<>(
r -> r.getHeader().hasError() ? buildFromPdpbError(r.getHeader().getError()) : null,
this);
ScatterRegionResponse resp =
callWithRetry(backOffer, PDGrpc.getScatterRegionMethod(), request, handler);
// TODO: maybe we should retry here, need dig into pd's codebase.
if (resp.hasHeader() && resp.getHeader().hasError()) {
throw new TiClientInternalException(
String.format("failed to scatter region because %s", resp.getHeader().getError()));
}
}
@Override
public TiRegion getRegionByKey(BackOffer backOffer, ByteString key) {
Supplier<GetRegionRequest> request;
@ -87,7 +126,8 @@ public class PDClient extends AbstractGRPCClient<PDBlockingStub, PDStub>
PDErrorHandler<GetRegionResponse> handler =
new PDErrorHandler<>(getRegionResponseErrorExtractor, this);
GetRegionResponse resp = callWithRetry(backOffer, PDGrpc.METHOD_GET_REGION, request, handler);
GetRegionResponse resp =
callWithRetry(backOffer, PDGrpc.getGetRegionMethod(), request, handler);
return new TiRegion(
resp.getRegion(),
resp.getLeader(),
@ -113,7 +153,7 @@ public class PDClient extends AbstractGRPCClient<PDBlockingStub, PDStub>
PDErrorHandler<GetRegionResponse> handler =
new PDErrorHandler<>(getRegionResponseErrorExtractor, this);
callAsyncWithRetry(backOffer, PDGrpc.METHOD_GET_REGION, request, responseObserver, handler);
callAsyncWithRetry(backOffer, PDGrpc.getGetRegionMethod(), request, responseObserver, handler);
return responseObserver.getFuture();
}
@ -125,7 +165,7 @@ public class PDClient extends AbstractGRPCClient<PDBlockingStub, PDStub>
new PDErrorHandler<>(getRegionResponseErrorExtractor, this);
GetRegionResponse resp =
callWithRetry(backOffer, PDGrpc.METHOD_GET_REGION_BY_ID, request, handler);
callWithRetry(backOffer, PDGrpc.getGetRegionByIDMethod(), request, handler);
// Instead of using default leader instance, explicitly set no leader to null
return new TiRegion(
resp.getRegion(),
@ -153,21 +193,28 @@ public class PDClient extends AbstractGRPCClient<PDBlockingStub, PDStub>
new PDErrorHandler<>(getRegionResponseErrorExtractor, this);
callAsyncWithRetry(
backOffer, PDGrpc.METHOD_GET_REGION_BY_ID, request, responseObserver, handler);
backOffer, PDGrpc.getGetRegionByIDMethod(), request, responseObserver, handler);
return responseObserver.getFuture();
}
private Supplier<GetStoreRequest> buildGetStoreReq(long storeId) {
return () -> GetStoreRequest.newBuilder().setHeader(header).setStoreId(storeId).build();
}
private Supplier<GetAllStoresRequest> buildGetAllStoresReq() {
return () -> GetAllStoresRequest.newBuilder().setHeader(header).build();
}
private <T> PDErrorHandler<GetStoreResponse> buildPDErrorHandler() {
return new PDErrorHandler<>(
r -> r.getHeader().hasError() ? buildFromPdpbError(r.getHeader().getError()) : null, this);
}
@Override
public Store getStore(BackOffer backOffer, long storeId) {
Supplier<GetStoreRequest> request =
() -> GetStoreRequest.newBuilder().setHeader(header).setStoreId(storeId).build();
PDErrorHandler<GetStoreResponse> handler =
new PDErrorHandler<>(
r -> r.getHeader().hasError() ? buildFromPdpbError(r.getHeader().getError()) : null,
this);
GetStoreResponse resp = callWithRetry(backOffer, PDGrpc.METHOD_GET_STORE, request, handler);
return resp.getStore();
return callWithRetry(
backOffer, PDGrpc.getGetStoreMethod(), buildGetStoreReq(storeId), buildPDErrorHandler())
.getStore();
}
@Override
@ -175,14 +222,12 @@ public class PDClient extends AbstractGRPCClient<PDBlockingStub, PDStub>
FutureObserver<Store, GetStoreResponse> responseObserver =
new FutureObserver<>(GetStoreResponse::getStore);
Supplier<GetStoreRequest> request =
() -> GetStoreRequest.newBuilder().setHeader(header).setStoreId(storeId).build();
PDErrorHandler<GetStoreResponse> handler =
new PDErrorHandler<>(
r -> r.getHeader().hasError() ? buildFromPdpbError(r.getHeader().getError()) : null,
this);
callAsyncWithRetry(backOffer, PDGrpc.METHOD_GET_STORE, request, responseObserver, handler);
callAsyncWithRetry(
backOffer,
PDGrpc.getGetStoreMethod(),
buildGetStoreReq(storeId),
responseObserver,
buildPDErrorHandler());
return responseObserver.getFuture();
}
@ -191,15 +236,11 @@ public class PDClient extends AbstractGRPCClient<PDBlockingStub, PDStub>
if (service != null) {
service.shutdownNow();
}
if (getLeaderWrapper() != null) {
getLeaderWrapper().close();
if (channelFactory != null) {
channelFactory.close();
}
}
public static ReadOnlyPDClient create(TiConfiguration conf, ChannelFactory channelFactory) {
return createRaw(conf, channelFactory);
}
@VisibleForTesting
RequestHeader getHeader() {
return header;
@ -210,56 +251,20 @@ public class PDClient extends AbstractGRPCClient<PDBlockingStub, PDStub>
return leaderWrapper;
}
class LeaderWrapper {
private final String leaderInfo;
private final PDBlockingStub blockingStub;
private final PDStub asyncStub;
private final long createTime;
LeaderWrapper(
String leaderInfo,
PDGrpc.PDBlockingStub blockingStub,
PDGrpc.PDStub asyncStub,
long createTime) {
this.leaderInfo = leaderInfo;
this.blockingStub = blockingStub;
this.asyncStub = asyncStub;
this.createTime = createTime;
}
String getLeaderInfo() {
return leaderInfo;
}
PDBlockingStub getBlockingStub() {
return blockingStub;
}
PDStub getAsyncStub() {
return asyncStub;
}
long getCreateTime() {
return createTime;
}
void close() {}
@Override
public String toString() {
return "[" + leaderInfo + "]";
}
}
public GetMembersResponse getMembers(URI url) {
private GetMembersResponse getMembers(URI url) {
try {
ManagedChannel probChan = channelFactory.getChannel(url.getHost() + ":" + url.getPort());
PDGrpc.PDBlockingStub stub = PDGrpc.newBlockingStub(probChan);
GetMembersRequest request =
GetMembersRequest.newBuilder().setHeader(RequestHeader.getDefaultInstance()).build();
return stub.getMembers(request);
GetMembersResponse resp = stub.getMembers(request);
// check if the response contains a valid leader
if (resp != null && resp.getLeader().getMemberId() == 0) {
return null;
}
return resp;
} catch (Exception e) {
logger.warn("failed to get member from pd server " + url + ".", e);
logger.warn("failed to get member from pd server.", e);
}
return null;
}
@ -277,7 +282,7 @@ public class PDClient extends AbstractGRPCClient<PDBlockingStub, PDStub>
private boolean createLeaderWrapper(String leaderUrlStr) {
try {
URI newLeader = URI.create(leaderUrlStr);
URI newLeader = PDUtils.addrToUrl(leaderUrlStr);
leaderUrlStr = newLeader.getHost() + ":" + newLeader.getPort();
if (leaderWrapper != null && leaderUrlStr.equals(leaderWrapper.getLeaderInfo())) {
return true;
@ -292,7 +297,7 @@ public class PDClient extends AbstractGRPCClient<PDBlockingStub, PDStub>
PDGrpc.newStub(clientChannel),
System.nanoTime());
} catch (IllegalArgumentException e) {
logger.error("Error updating leader.", e);
logger.error("Error updating leader. " + leaderUrlStr, e);
return false;
}
logger.info(String.format("Switched to new leader: %s", leaderWrapper));
@ -335,10 +340,6 @@ public class PDClient extends AbstractGRPCClient<PDBlockingStub, PDStub>
.withDeadlineAfter(getConf().getTimeout(), getConf().getTimeoutUnit());
}
private PDClient(TiConfiguration conf, ChannelFactory channelFactory) {
super(conf, channelFactory);
}
private void initCluster() {
GetMembersResponse resp = null;
List<URI> pdAddrs = getConf().getPdAddrs();
@ -371,20 +372,42 @@ public class PDClient extends AbstractGRPCClient<PDBlockingStub, PDStub>
TimeUnit.MINUTES);
}
static PDClient createRaw(TiConfiguration conf, ChannelFactory channelFactory) {
PDClient client = null;
try {
client = new PDClient(conf, channelFactory);
client.initCluster();
} catch (Exception e) {
if (client != null) {
try {
client.close();
} catch (TiKVException ignore) {
}
}
throw e;
static class LeaderWrapper {
private final String leaderInfo;
private final PDBlockingStub blockingStub;
private final PDStub asyncStub;
private final long createTime;
LeaderWrapper(
String leaderInfo,
PDGrpc.PDBlockingStub blockingStub,
PDGrpc.PDStub asyncStub,
long createTime) {
this.leaderInfo = leaderInfo;
this.blockingStub = blockingStub;
this.asyncStub = asyncStub;
this.createTime = createTime;
}
String getLeaderInfo() {
return leaderInfo;
}
PDBlockingStub getBlockingStub() {
return blockingStub;
}
PDStub getAsyncStub() {
return asyncStub;
}
long getCreateTime() {
return createTime;
}
@Override
public String toString() {
return "[leaderInfo: " + leaderInfo + "]";
}
return client;
}
}

View File

@ -0,0 +1,61 @@
/*
* Copyright 2020 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.tikv.common;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class StoreVersion {
private static final int SCALE = 10000;
private final Logger logger = LoggerFactory.getLogger(this.getClass());
private int v0 = 9999;
private int v1 = 9999;
private int v2 = 9999;
private StoreVersion(String version) {
try {
// tiflash version starts with `v`
if (version.startsWith("v")) {
version = version.substring(1);
}
String[] parts = version.split("[.-]");
if (parts.length > 0) {
v0 = Integer.parseInt(parts[0]);
}
if (parts.length > 1) {
v1 = Integer.parseInt(parts[1]);
}
if (parts.length > 2) {
v2 = Integer.parseInt(parts[2]);
}
} catch (Exception e) {
logger.warn("invalid store version: " + version, e);
}
}
public static int compareTo(String v0, String v1) {
return new StoreVersion(v0).toIntVersion() - new StoreVersion(v1).toIntVersion();
}
private int toIntVersion() {
return v0 * SCALE * SCALE + v1 * SCALE + v2;
}
private boolean greatThan(StoreVersion other) {
return toIntVersion() > other.toIntVersion();
}
}

View File

@ -15,13 +15,14 @@
package org.tikv.common;
import com.google.common.collect.ImmutableList;
import java.io.Serializable;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
import org.tikv.common.pd.PDUtils;
import org.tikv.kvproto.Kvrpcpb.CommandPri;
import org.tikv.kvproto.Kvrpcpb.IsolationLevel;
@ -31,19 +32,24 @@ public class TiConfiguration implements Serializable {
private static final int DEF_SCAN_BATCH_SIZE = 100;
private static final boolean DEF_IGNORE_TRUNCATE = true;
private static final boolean DEF_TRUNCATE_AS_WARNING = false;
private static final int DEF_META_RELOAD_PERIOD = 10;
private static final TimeUnit DEF_META_RELOAD_UNIT = TimeUnit.SECONDS;
private static final int DEF_MAX_FRAME_SIZE = 268435456 * 2; // 256 * 2 MB
private static final int DEF_INDEX_SCAN_BATCH_SIZE = 20000;
private static final int DEF_REGION_SCAN_DOWNGRADE_THRESHOLD = 10000000;
// if keyRange size per request exceeds this limit, the request might be too large to be accepted
// by TiKV(maximum request size accepted by TiKV is around 1MB)
private static final int MAX_REQUEST_KEY_RANGE_SIZE = 20000;
private static final int DEF_INDEX_SCAN_CONCURRENCY = 5;
private static final int DEF_TABLE_SCAN_CONCURRENCY = 512;
private static final CommandPri DEF_COMMAND_PRIORITY = CommandPri.Low;
private static final IsolationLevel DEF_ISOLATION_LEVEL = IsolationLevel.RC;
private static final IsolationLevel DEF_ISOLATION_LEVEL = IsolationLevel.SI;
private static final boolean DEF_SHOW_ROWID = false;
private static final String DEF_DB_PREFIX = "";
private static final boolean DEF_WRITE_ENABLE = true;
private static final boolean DEF_WRITE_ALLOW_SPARK_SQL = false;
private static final boolean DEF_WRITE_WITHOUT_LOCK_TABLE = false;
private static final int DEF_TIKV_REGION_SPLIT_SIZE_IN_MB = 96;
private static final int DEF_PARTITION_PER_SPLIT = 1;
private static final int DEF_KV_CLIENT_CONCURRENCY = 10;
private static final KVMode DEF_KV_MODE = KVMode.TXN;
private static final int DEF_RAW_CLIENT_CONCURRENCY = 200;
@ -51,11 +57,10 @@ public class TiConfiguration implements Serializable {
private TimeUnit timeoutUnit = DEF_TIMEOUT_UNIT;
private boolean ignoreTruncate = DEF_IGNORE_TRUNCATE;
private boolean truncateAsWarning = DEF_TRUNCATE_AS_WARNING;
private TimeUnit metaReloadUnit = DEF_META_RELOAD_UNIT;
private int metaReloadPeriod = DEF_META_RELOAD_PERIOD;
private int maxFrameSize = DEF_MAX_FRAME_SIZE;
private List<URI> pdAddrs = new ArrayList<>();
private int indexScanBatchSize = DEF_INDEX_SCAN_BATCH_SIZE;
private int downgradeThreshold = DEF_REGION_SCAN_DOWNGRADE_THRESHOLD;
private int indexScanConcurrency = DEF_INDEX_SCAN_CONCURRENCY;
private int tableScanConcurrency = DEF_TABLE_SCAN_CONCURRENCY;
private CommandPri commandPriority = DEF_COMMAND_PRIORITY;
@ -66,6 +71,14 @@ public class TiConfiguration implements Serializable {
private KVMode kvMode = DEF_KV_MODE;
private int rawClientConcurrency = DEF_RAW_CLIENT_CONCURRENCY;
private boolean writeAllowSparkSQL = DEF_WRITE_ALLOW_SPARK_SQL;
private boolean writeEnable = DEF_WRITE_ENABLE;
private boolean writeWithoutLockTable = DEF_WRITE_WITHOUT_LOCK_TABLE;
private int tikvRegionSplitSizeInMB = DEF_TIKV_REGION_SPLIT_SIZE_IN_MB;
private int partitionPerSplit = DEF_PARTITION_PER_SPLIT;
private int kvClientConcurrency = DEF_KV_CLIENT_CONCURRENCY;
public enum KVMode {
TXN,
RAW
@ -89,11 +102,21 @@ public class TiConfiguration implements Serializable {
private static List<URI> strToURI(String addressStr) {
Objects.requireNonNull(addressStr);
String[] addrs = addressStr.split(",");
ImmutableList.Builder<URI> addrsBuilder = ImmutableList.builder();
for (String addr : addrs) {
addrsBuilder.add(URI.create("http://" + addr));
Arrays.sort(addrs);
return PDUtils.addrsToUrls(addrs);
}
public static <E> String listToString(List<E> list) {
StringBuilder sb = new StringBuilder();
sb.append("[");
for (int i = 0; i < list.size(); i++) {
sb.append(list.get(i).toString());
if (i != list.size() - 1) {
sb.append(",");
}
}
return addrsBuilder.build();
sb.append("]");
return sb.toString();
}
public int getTimeout() {
@ -109,24 +132,6 @@ public class TiConfiguration implements Serializable {
return timeoutUnit;
}
public TimeUnit getMetaReloadPeriodUnit() {
return metaReloadUnit;
}
public TiConfiguration setMetaReloadPeriodUnit(TimeUnit timeUnit) {
this.metaReloadUnit = timeUnit;
return this;
}
public TiConfiguration setMetaReloadPeriod(int metaReloadPeriod) {
this.metaReloadPeriod = metaReloadPeriod;
return this;
}
public int getMetaReloadPeriod() {
return metaReloadPeriod;
}
public TiConfiguration setTimeoutUnit(TimeUnit timeoutUnit) {
this.timeoutUnit = timeoutUnit;
return this;
@ -136,6 +141,10 @@ public class TiConfiguration implements Serializable {
return pdAddrs;
}
public String getPdAddrsString() {
return listToString(pdAddrs);
}
public int getScanBatchSize() {
return DEF_SCAN_BATCH_SIZE;
}
@ -234,6 +243,54 @@ public class TiConfiguration implements Serializable {
this.dbPrefix = dbPrefix;
}
public boolean isWriteEnable() {
return writeEnable;
}
public void setWriteEnable(boolean writeEnable) {
this.writeEnable = writeEnable;
}
public boolean isWriteWithoutLockTable() {
return writeWithoutLockTable;
}
public void setWriteWithoutLockTable(boolean writeWithoutLockTable) {
this.writeWithoutLockTable = writeWithoutLockTable;
}
public boolean isWriteAllowSparkSQL() {
return writeAllowSparkSQL;
}
public void setWriteAllowSparkSQL(boolean writeAllowSparkSQL) {
this.writeAllowSparkSQL = writeAllowSparkSQL;
}
public int getTikvRegionSplitSizeInMB() {
return tikvRegionSplitSizeInMB;
}
public void setTikvRegionSplitSizeInMB(int tikvRegionSplitSizeInMB) {
this.tikvRegionSplitSizeInMB = tikvRegionSplitSizeInMB;
}
public int getDowngradeThreshold() {
return downgradeThreshold;
}
public void setDowngradeThreshold(int downgradeThreshold) {
this.downgradeThreshold = downgradeThreshold;
}
public int getPartitionPerSplit() {
return partitionPerSplit;
}
public void setPartitionPerSplit(int partitionPerSplit) {
this.partitionPerSplit = partitionPerSplit;
}
public KVMode getKvMode() {
return kvMode;
}
@ -249,4 +306,12 @@ public class TiConfiguration implements Serializable {
public void setRawClientConcurrency(int rawClientConcurrency) {
this.rawClientConcurrency = rawClientConcurrency;
}
public int getKvClientConcurrency() {
return kvClientConcurrency;
}
public void setKvClientConcurrency(int kvClientConcurrency) {
this.kvClientConcurrency = kvClientConcurrency;
}
}

View File

@ -20,6 +20,7 @@ import org.tikv.common.region.RegionManager;
import org.tikv.common.region.RegionStoreClient.RegionStoreClientBuilder;
import org.tikv.common.util.ChannelFactory;
import org.tikv.raw.RawKVClient;
import org.tikv.txn.KVClient;
/**
* TiSession is the holder for PD Client, Store pdClient and PD Cache All sessions share common
@ -49,10 +50,23 @@ public class TiSession implements AutoCloseable {
// Create new Region Manager avoiding thread contentions
RegionManager regionMgr = new RegionManager(pdClient);
RegionStoreClientBuilder builder =
new RegionStoreClientBuilder(conf, channelFactory, regionMgr);
new RegionStoreClientBuilder(conf, channelFactory, regionMgr, pdClient);
return new RawKVClient(conf, builder);
}
public KVClient createTxnKVClient() {
// Create new Region Manager avoiding thread contentions
RegionManager regionMgr = new RegionManager(pdClient);
RegionStoreClientBuilder builder =
new RegionStoreClientBuilder(conf, channelFactory, regionMgr, pdClient);
return new KVClient(conf, builder);
}
@VisibleForTesting
public RegionManager getRegionManager() {
return new RegionManager(pdClient);
}
@VisibleForTesting
public PDClient getPDClient() {
return pdClient;

View File

@ -0,0 +1,28 @@
/*
*
* Copyright 2020 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*
*/
package org.tikv.common;
public class Version {
public static final String RESOLVE_LOCK_V2 = "2.0.0";
public static final String RESOLVE_LOCK_V3 = "3.0.5";
public static final String RESOLVE_LOCK_V4 = "4.0.0";
public static final String BATCH_WRITE = "3.0.14";
}

View File

@ -17,6 +17,7 @@ package org.tikv.common.codec;
import com.google.common.primitives.UnsignedBytes;
import com.google.protobuf.ByteString;
import com.google.protobuf.TextFormat;
import org.tikv.kvproto.Coprocessor;
public class KeyUtils {
@ -40,10 +41,31 @@ public class KeyUtils {
}
public static String formatBytes(Coprocessor.KeyRange keyRange) {
return "[[" + formatBytes(keyRange.getStart()) + "], [" + formatBytes(keyRange.getEnd()) + "])";
return "([" + formatBytes(keyRange.getStart()) + "], [" + formatBytes(keyRange.getEnd()) + "])";
}
public static String formatBytesUTF8(byte[] bytes) {
if (bytes == null) return "null";
return TextFormat.escapeBytes(bytes);
}
public static String formatBytesUTF8(ByteString bytes) {
if (bytes == null) return "null";
return formatBytesUTF8(bytes.toByteArray());
}
public static String formatBytesUTF8(Coprocessor.KeyRange keyRange) {
return "(["
+ formatBytesUTF8(keyRange.getStart())
+ "], ["
+ formatBytesUTF8(keyRange.getEnd())
+ "])";
}
public static boolean hasPrefix(ByteString str, ByteString prefix) {
if (prefix.size() > str.size()) {
return false;
}
for (int i = 0; i < prefix.size(); i++) {
if (str.byteAt(i) != prefix.byteAt(i)) {
return false;

View File

@ -16,6 +16,7 @@
package org.tikv.common.meta;
import java.io.Serializable;
import java.util.Objects;
/** TiTimestamp is the timestamp returned by timestamp oracle inside placement driver */
public class TiTimestamp implements Serializable {
@ -29,6 +30,10 @@ public class TiTimestamp implements Serializable {
this.logical = l;
}
public static long extractPhysical(long ts) {
return ts >> PHYSICAL_SHIFT_BITS;
}
public long getVersion() {
return (physical << PHYSICAL_SHIFT_BITS) + logical;
}
@ -40,4 +45,29 @@ public class TiTimestamp implements Serializable {
public long getLogical() {
return this.logical;
}
public TiTimestamp getPrevious() {
return new TiTimestamp(physical, logical - 1);
}
@Override
public boolean equals(Object other) {
if (other == this) {
return true;
}
if (!(other instanceof TiTimestamp)) {
return false;
}
return this.getVersion() == ((TiTimestamp) other).getVersion();
}
@Override
public int hashCode() {
return Objects.hash(getVersion());
}
@Override
public String toString() {
return "TiTimestamp(" + getVersion() + ")";
}
}

View File

@ -17,29 +17,64 @@
package org.tikv.common.operation;
import static org.tikv.common.util.BackOffFunction.BackOffFuncType.BoTxnLockFast;
import com.google.protobuf.ByteString;
import io.grpc.Status;
import io.grpc.StatusRuntimeException;
import java.util.Collections;
import java.util.function.Function;
import org.apache.log4j.Logger;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.tikv.common.codec.KeyUtils;
import org.tikv.common.exception.GrpcException;
import org.tikv.common.exception.KeyException;
import org.tikv.common.region.RegionErrorReceiver;
import org.tikv.common.region.RegionManager;
import org.tikv.common.region.TiRegion;
import org.tikv.common.util.BackOffFunction;
import org.tikv.common.util.BackOffer;
import org.tikv.kvproto.Errorpb;
import org.tikv.kvproto.Kvrpcpb;
import org.tikv.txn.AbstractLockResolverClient;
import org.tikv.txn.Lock;
import org.tikv.txn.ResolveLockResult;
// TODO: consider refactor to Builder mode
public class KVErrorHandler<RespT> implements ErrorHandler<RespT> {
private static final Logger logger = Logger.getLogger(KVErrorHandler.class);
private static final int NO_LEADER_STORE_ID =
0; // if there's currently no leader of a store, store id is set to 0
private static final Logger logger = LoggerFactory.getLogger(KVErrorHandler.class);
// if a store does not have leader currently, store id is set to 0
private static final int NO_LEADER_STORE_ID = 0;
private final Function<RespT, Errorpb.Error> getRegionError;
private final Function<RespT, Kvrpcpb.KeyError> getKeyError;
private final Function<ResolveLockResult, Object> resolveLockResultCallback;
private final RegionManager regionManager;
private final RegionErrorReceiver recv;
private final AbstractLockResolverClient lockResolverClient;
private final TiRegion ctxRegion;
private final long callerStartTS;
private final boolean forWrite;
public KVErrorHandler(
RegionManager regionManager,
RegionErrorReceiver recv,
AbstractLockResolverClient lockResolverClient,
TiRegion ctxRegion,
Function<RespT, Errorpb.Error> getRegionError,
Function<RespT, Kvrpcpb.KeyError> getKeyError,
Function<ResolveLockResult, Object> resolveLockResultCallback,
long callerStartTS,
boolean forWrite) {
this.ctxRegion = ctxRegion;
this.recv = recv;
this.lockResolverClient = lockResolverClient;
this.regionManager = regionManager;
this.getRegionError = getRegionError;
this.getKeyError = getKeyError;
this.resolveLockResultCallback = resolveLockResultCallback;
this.callerStartTS = callerStartTS;
this.forWrite = forWrite;
}
public KVErrorHandler(
RegionManager regionManager,
@ -48,8 +83,13 @@ public class KVErrorHandler<RespT> implements ErrorHandler<RespT> {
Function<RespT, Errorpb.Error> getRegionError) {
this.ctxRegion = ctxRegion;
this.recv = recv;
this.lockResolverClient = null;
this.regionManager = regionManager;
this.getRegionError = getRegionError;
this.getKeyError = resp -> null;
this.resolveLockResultCallback = resolveLock -> null;
this.callerStartTS = 0;
this.forWrite = false;
}
private Errorpb.Error getRegionError(RespT resp) {
@ -64,8 +104,27 @@ public class KVErrorHandler<RespT> implements ErrorHandler<RespT> {
regionManager.invalidateStore(ctxRegion.getLeader().getStoreId());
}
private void resolveLock(BackOffer backOffer, Lock lock) {
if (lockResolverClient != null) {
logger.warn("resolving lock");
ResolveLockResult resolveLockResult =
lockResolverClient.resolveLocks(
backOffer, callerStartTS, Collections.singletonList(lock), forWrite);
resolveLockResultCallback.apply(resolveLockResult);
long msBeforeExpired = resolveLockResult.getMsBeforeTxnExpired();
if (msBeforeExpired > 0) {
// if not resolve all locks, we wait and retry
backOffer.doBackOffWithMaxSleep(
BoTxnLockFast, msBeforeExpired, new KeyException(lock.toString()));
}
}
}
// Referenced from TiDB
// store/tikv/region_request.go - onRegionError
/** @return true: client should retry */
@Override
public boolean handleResponseError(BackOffer backOffer, RespT resp) {
if (resp == null) {
@ -96,16 +155,17 @@ public class KVErrorHandler<RespT> implements ErrorHandler<RespT> {
// if there's current no leader, we do not trigger update pd cache logic
// since issuing store = NO_LEADER_STORE_ID requests to pd will definitely fail.
if (newStoreId != NO_LEADER_STORE_ID) {
if (!this.regionManager.checkAndDropLeader(ctxRegion.getId(), newStoreId)
if (!this.regionManager.updateLeader(ctxRegion.getId(), newStoreId)
|| !recv.onNotLeader(this.regionManager.getStoreById(newStoreId))) {
// If update leader fails, we need to fetch new region info from pd,
// and re-split key range for new region. Setting retry to false will
// stop retry and enter handleCopResponse logic, which would use RegionMiss
// backOff strategy to wait, fetch new region and re-split key range.
// onNotLeader is only needed when checkAndDropLeader succeeds, thus switch
// onNotLeader is only needed when updateLeader succeeds, thus switch
// to a new store address.
retry = false;
}
backOffFuncType = BackOffFunction.BackOffFuncType.BoUpdateLeader;
} else {
logger.info(
@ -113,7 +173,9 @@ public class KVErrorHandler<RespT> implements ErrorHandler<RespT> {
"Received zero store id, from region %d try next time", ctxRegion.getId()));
backOffFuncType = BackOffFunction.BackOffFuncType.BoRegionMiss;
}
backOffer.doBackOff(backOffFuncType, new GrpcException(error.toString()));
return retry;
} else if (error.hasStoreNotMatch()) {
// this error is reported from raftstore:
@ -124,12 +186,9 @@ public class KVErrorHandler<RespT> implements ErrorHandler<RespT> {
String.format(
"Store Not Match happened with region id %d, store id %d",
ctxRegion.getId(), storeId));
logger.warn(String.format("%s", error.getStoreNotMatch()));
this.regionManager.invalidateStore(storeId);
recv.onStoreNotMatch(this.regionManager.getStoreById(storeId));
backOffer.doBackOff(
BackOffFunction.BackOffFuncType.BoStoreNotMatch, new GrpcException(error.toString()));
return true;
} else if (error.hasEpochNotMatch()) {
// this error is reported from raftstore:
@ -166,17 +225,30 @@ public class KVErrorHandler<RespT> implements ErrorHandler<RespT> {
logger.error(
String.format(
"Key not in region [%s] for key [%s], this error should not happen here.",
ctxRegion, KeyUtils.formatBytes(invalidKey)));
ctxRegion, KeyUtils.formatBytesUTF8(invalidKey)));
throw new StatusRuntimeException(Status.UNKNOWN.withDescription(error.toString()));
}
logger.warn(String.format("Unknown error for region [%s]", ctxRegion));
logger.warn(String.format("Unknown error %s for region [%s]", error.toString(), ctxRegion));
// For other errors, we only drop cache here.
// Upper level may split this task.
invalidateRegionStoreCache(ctxRegion);
}
return false;
boolean retry = false;
// Key error handling logic
Kvrpcpb.KeyError keyError = getKeyError.apply(resp);
if (keyError != null) {
try {
Lock lock = AbstractLockResolverClient.extractLockFromKeyErr(keyError);
resolveLock(backOffer, lock);
retry = true;
} catch (KeyException e) {
logger.warn("Unable to handle KeyExceptions other than LockException", e);
}
}
return retry;
}
@Override
@ -187,6 +259,8 @@ public class KVErrorHandler<RespT> implements ErrorHandler<RespT> {
BackOffFunction.BackOffFuncType.BoTiKVRPC,
new GrpcException(
"send tikv request error: " + e.getMessage() + ", try next peer later", e));
return true;
// TiKV maybe down, so do not retry in `callWithRetry`
// should re-fetch the new leader from PD and send request to it
return false;
}
}

View File

@ -20,7 +20,8 @@ package org.tikv.common.operation;
import static org.tikv.common.pd.PDError.buildFromPdpbError;
import java.util.function.Function;
import org.apache.log4j.Logger;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.tikv.common.PDClient;
import org.tikv.common.exception.GrpcException;
import org.tikv.common.exception.TiClientInternalException;
@ -30,15 +31,14 @@ import org.tikv.common.util.BackOffer;
import org.tikv.kvproto.Pdpb;
public class PDErrorHandler<RespT> implements ErrorHandler<RespT> {
private static final Logger logger = Logger.getLogger(PDErrorHandler.class);
private final Function<RespT, PDError> getError;
private final PDClient client;
public static final Function<Pdpb.GetRegionResponse, PDError> getRegionResponseErrorExtractor =
r ->
r.getHeader().hasError()
? buildFromPdpbError(r.getHeader().getError())
: r.getRegion().getId() == 0 ? PDError.RegionPeerNotElected.DEFAULT_INSTANCE : null;
private static final Logger logger = LoggerFactory.getLogger(PDErrorHandler.class);
private final Function<RespT, PDError> getError;
private final PDClient client;
public PDErrorHandler(Function<RespT, PDError> errorExtractor, PDClient client) {
this.getError = errorExtractor;
@ -54,12 +54,12 @@ public class PDErrorHandler<RespT> implements ErrorHandler<RespT> {
if (error != null) {
switch (error.getErrorType()) {
case PD_ERROR:
client.updateLeader();
backOffer.doBackOff(
BackOffFunction.BackOffFuncType.BoPDRPC, new GrpcException(error.toString()));
client.updateLeader();
return true;
case REGION_PEER_NOT_ELECTED:
logger.info(error.getMessage());
logger.debug(error.getMessage());
backOffer.doBackOff(
BackOffFunction.BackOffFuncType.BoPDRPC, new GrpcException(error.toString()));
return true;

View File

@ -15,25 +15,61 @@
package org.tikv.common.operation.iterator;
import static java.util.Objects.requireNonNull;
import com.google.protobuf.ByteString;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.tikv.common.TiConfiguration;
import org.tikv.common.exception.GrpcException;
import org.tikv.common.exception.KeyException;
import org.tikv.common.key.Key;
import org.tikv.common.region.RegionStoreClient;
import org.tikv.common.region.RegionStoreClient.RegionStoreClientBuilder;
import org.tikv.common.region.TiRegion;
import org.tikv.common.util.BackOffer;
import org.tikv.common.util.ConcreteBackOffer;
import org.tikv.common.util.Pair;
import org.tikv.kvproto.Kvrpcpb;
import org.tikv.kvproto.Metapb;
public class ConcreteScanIterator extends ScanIterator {
private final long version;
private final Logger logger = LoggerFactory.getLogger(ConcreteScanIterator.class);
public ConcreteScanIterator(
TiConfiguration conf, RegionStoreClientBuilder builder, ByteString startKey, long version) {
TiConfiguration conf,
RegionStoreClientBuilder builder,
ByteString startKey,
long version,
int limit) {
// Passing endKey as ByteString.EMPTY means that endKey is +INF by default,
super(conf, builder, startKey, ByteString.EMPTY, Integer.MAX_VALUE);
this(conf, builder, startKey, ByteString.EMPTY, version, limit);
}
public ConcreteScanIterator(
TiConfiguration conf,
RegionStoreClientBuilder builder,
ByteString startKey,
ByteString endKey,
long version) {
// Passing endKey as ByteString.EMPTY means that endKey is +INF by default,
this(conf, builder, startKey, endKey, version, Integer.MAX_VALUE);
}
private ConcreteScanIterator(
TiConfiguration conf,
RegionStoreClientBuilder builder,
ByteString startKey,
ByteString endKey,
long version,
int limit) {
super(conf, builder, startKey, endKey, limit);
this.version = version;
}
TiRegion loadCurrentRegionToCache() throws Exception {
@Override
TiRegion loadCurrentRegionToCache() throws GrpcException {
TiRegion region;
try (RegionStoreClient client = builder.build(startKey)) {
region = client.getRegion();
@ -42,4 +78,66 @@ public class ConcreteScanIterator extends ScanIterator {
return region;
}
}
private ByteString resolveCurrentLock(Kvrpcpb.KvPair current) {
logger.warn(String.format("resolve current key error %s", current.getError().toString()));
Pair<TiRegion, Metapb.Store> pair =
builder.getRegionManager().getRegionStorePairByKey(current.getKey());
TiRegion region = pair.first;
Metapb.Store store = pair.second;
BackOffer backOffer = ConcreteBackOffer.newGetBackOff();
try (RegionStoreClient client = builder.build(region, store)) {
return client.get(backOffer, current.getKey(), version);
} catch (Exception e) {
throw new KeyException(current.getError());
}
}
@Override
public boolean hasNext() {
Kvrpcpb.KvPair current;
// continue when cache is empty but not null
do {
current = getCurrent();
if (isCacheDrained() && cacheLoadFails()) {
endOfScan = true;
return false;
}
} while (currentCache != null && current == null);
// for last batch to be processed, we have to check if
return !processingLastBatch
|| current == null
|| (hasEndKey && Key.toRawKey(current.getKey()).compareTo(endKey) < 0);
}
@Override
public Kvrpcpb.KvPair next() {
--limit;
Kvrpcpb.KvPair current = currentCache.get(index++);
requireNonNull(current, "current kv pair cannot be null");
if (current.hasError()) {
ByteString val = resolveCurrentLock(current);
current = Kvrpcpb.KvPair.newBuilder().setKey(current.getKey()).setValue(val).build();
}
return current;
}
/**
* Cache is drained when - no data extracted - scan limit was not defined - have read the last
* index of cache - index not initialized
*
* @return whether cache is drained
*/
private boolean isCacheDrained() {
return currentCache == null || limit <= 0 || index >= currentCache.size() || index == -1;
}
private Kvrpcpb.KvPair getCurrent() {
if (isCacheDrained()) {
return null;
}
return currentCache.get(index);
}
}

View File

@ -17,6 +17,7 @@ package org.tikv.common.operation.iterator;
import com.google.protobuf.ByteString;
import org.tikv.common.TiConfiguration;
import org.tikv.common.exception.GrpcException;
import org.tikv.common.exception.TiKVException;
import org.tikv.common.key.Key;
import org.tikv.common.region.RegionStoreClient;
@ -25,6 +26,7 @@ import org.tikv.common.region.TiRegion;
import org.tikv.common.util.BackOffFunction;
import org.tikv.common.util.BackOffer;
import org.tikv.common.util.ConcreteBackOffer;
import org.tikv.kvproto.Kvrpcpb;
public class RawScanIterator extends ScanIterator {
@ -37,7 +39,7 @@ public class RawScanIterator extends ScanIterator {
super(conf, builder, startKey, endKey, limit);
}
TiRegion loadCurrentRegionToCache() throws Exception {
TiRegion loadCurrentRegionToCache() throws GrpcException {
BackOffer backOffer = ConcreteBackOffer.newScannerNextMaxBackOff();
while (true) {
try (RegionStoreClient client = builder.build(startKey)) {
@ -59,11 +61,15 @@ public class RawScanIterator extends ScanIterator {
private boolean notEndOfScan() {
return limit > 0
&& !(lastBatch
&& !(processingLastBatch
&& (index >= currentCache.size()
|| Key.toRawKey(currentCache.get(index).getKey()).compareTo(endKey) >= 0));
}
boolean isCacheDrained() {
return currentCache == null || limit <= 0 || index >= currentCache.size() || index == -1;
}
@Override
public boolean hasNext() {
if (isCacheDrained() && cacheLoadFails()) {
@ -72,4 +78,24 @@ public class RawScanIterator extends ScanIterator {
}
return notEndOfScan();
}
private Kvrpcpb.KvPair getCurrent() {
if (isCacheDrained()) {
return null;
}
--limit;
return currentCache.get(index++);
}
@Override
public Kvrpcpb.KvPair next() {
Kvrpcpb.KvPair kv;
// continue when cache is empty but not null
for (kv = getCurrent(); currentCache != null && kv == null; kv = getCurrent()) {
if (cacheLoadFails()) {
return null;
}
}
return kv;
}
}

View File

@ -21,6 +21,7 @@ import com.google.protobuf.ByteString;
import java.util.Iterator;
import java.util.List;
import org.tikv.common.TiConfiguration;
import org.tikv.common.exception.GrpcException;
import org.tikv.common.exception.TiClientInternalException;
import org.tikv.common.key.Key;
import org.tikv.common.region.RegionStoreClient.RegionStoreClientBuilder;
@ -38,7 +39,7 @@ public abstract class ScanIterator implements Iterator<Kvrpcpb.KvPair> {
protected Key endKey;
protected boolean hasEndKey;
protected boolean lastBatch = false;
protected boolean processingLastBatch = false;
ScanIterator(
TiConfiguration conf,
@ -57,14 +58,20 @@ public abstract class ScanIterator implements Iterator<Kvrpcpb.KvPair> {
this.builder = builder;
}
abstract TiRegion loadCurrentRegionToCache() throws Exception;
/**
* Load current region to cache, returns the region if loaded.
*
* @return TiRegion of current data loaded to cache
* @throws GrpcException if scan still fails after backoff
*/
abstract TiRegion loadCurrentRegionToCache() throws GrpcException;
// return true if current cache is not loaded or empty
boolean cacheLoadFails() {
if (endOfScan || lastBatch) {
if (endOfScan || processingLastBatch) {
return true;
}
if (startKey.isEmpty()) {
if (startKey == null || startKey.isEmpty()) {
return true;
}
try {
@ -84,6 +91,13 @@ public abstract class ScanIterator implements Iterator<Kvrpcpb.KvPair> {
// of a transaction. Otherwise below code might lose data
if (currentCache.size() < conf.getScanBatchSize()) {
startKey = curRegionEndKey;
lastKey = Key.toRawKey(curRegionEndKey);
} else if (currentCache.size() > conf.getScanBatchSize()) {
throw new IndexOutOfBoundsException(
"current cache size = "
+ currentCache.size()
+ ", larger than "
+ conf.getScanBatchSize());
} else {
// Start new scan from exact next key in current region
lastKey = Key.toRawKey(currentCache.get(currentCache.size() - 1).getKey());
@ -91,7 +105,7 @@ public abstract class ScanIterator implements Iterator<Kvrpcpb.KvPair> {
}
// notify last batch if lastKey is greater than or equal to endKey
if (hasEndKey && lastKey.compareTo(endKey) >= 0) {
lastBatch = true;
processingLastBatch = true;
startKey = null;
}
} catch (Exception e) {
@ -99,37 +113,4 @@ public abstract class ScanIterator implements Iterator<Kvrpcpb.KvPair> {
}
return false;
}
boolean isCacheDrained() {
return currentCache == null || limit <= 0 || index >= currentCache.size() || index == -1;
}
@Override
public boolean hasNext() {
if (isCacheDrained() && cacheLoadFails()) {
endOfScan = true;
return false;
}
return true;
}
private Kvrpcpb.KvPair getCurrent() {
if (isCacheDrained()) {
return null;
}
--limit;
return currentCache.get(index++);
}
@Override
public Kvrpcpb.KvPair next() {
Kvrpcpb.KvPair kv;
// continue when cache is empty but not null
for (kv = getCurrent(); currentCache != null && kv == null; kv = getCurrent()) {
if (cacheLoadFails()) {
return null;
}
}
return kv;
}
}

View File

@ -24,11 +24,6 @@ public final class PDError {
private final ErrorType errorType;
public enum ErrorType {
PD_ERROR,
REGION_PEER_NOT_ELECTED
}
private PDError(Pdpb.Error error) {
this.error = error;
this.errorType = ErrorType.PD_ERROR;
@ -68,6 +63,11 @@ public final class PDError {
return "\nErrorType: " + errorType + "\nError: " + error;
}
public enum ErrorType {
PD_ERROR,
REGION_PEER_NOT_ELECTED
}
public static final class RegionPeerNotElected {
private static final String ERROR_MESSAGE = "Region Peer not elected. Please try later";
private static final Pdpb.Error DEFAULT_ERROR =

View File

@ -0,0 +1,38 @@
/*
* Copyright 2019 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.tikv.common.pd;
import com.google.common.collect.ImmutableList;
import java.net.URI;
import java.util.List;
public class PDUtils {
public static URI addrToUrl(String addr) {
if (addr.contains("://")) {
return URI.create(addr);
} else {
return URI.create("http://" + addr);
}
}
public static List<URI> addrsToUrls(String[] addrs) {
ImmutableList.Builder<URI> urlsBuilder = new ImmutableList.Builder<>();
for (String addr : addrs) {
urlsBuilder.add(addrToUrl(addr));
}
return urlsBuilder.build();
}
}

View File

@ -0,0 +1,112 @@
/*
*
* Copyright 2019 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*
*/
package org.tikv.common.region;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import io.grpc.ManagedChannel;
import org.tikv.common.AbstractGRPCClient;
import org.tikv.common.TiConfiguration;
import org.tikv.common.exception.GrpcException;
import org.tikv.common.util.ChannelFactory;
import org.tikv.kvproto.Metapb;
import org.tikv.kvproto.TikvGrpc;
public abstract class AbstractRegionStoreClient
extends AbstractGRPCClient<TikvGrpc.TikvBlockingStub, TikvGrpc.TikvStub>
implements RegionErrorReceiver {
protected final RegionManager regionManager;
protected TiRegion region;
protected AbstractRegionStoreClient(
TiConfiguration conf,
TiRegion region,
ChannelFactory channelFactory,
TikvGrpc.TikvBlockingStub blockingStub,
TikvGrpc.TikvStub asyncStub,
RegionManager regionManager) {
super(conf, channelFactory, blockingStub, asyncStub);
checkNotNull(region, "Region is empty");
checkNotNull(region.getLeader(), "Leader Peer is null");
checkArgument(region.getLeader() != null, "Leader Peer is null");
this.region = region;
this.regionManager = regionManager;
}
public TiRegion getRegion() {
return region;
}
@Override
protected TikvGrpc.TikvBlockingStub getBlockingStub() {
return blockingStub.withDeadlineAfter(getConf().getTimeout(), getConf().getTimeoutUnit());
}
@Override
protected TikvGrpc.TikvStub getAsyncStub() {
return asyncStub.withDeadlineAfter(getConf().getTimeout(), getConf().getTimeoutUnit());
}
@Override
public void close() throws GrpcException {}
/**
* onNotLeader deals with NotLeaderError and returns whether re-splitting key range is needed
*
* @param newStore the new store presented by NotLeader Error
* @return false when re-split is needed.
*/
@Override
public boolean onNotLeader(Metapb.Store newStore) {
if (logger.isDebugEnabled()) {
logger.debug(region + ", new leader = " + newStore.getId());
}
TiRegion cachedRegion = regionManager.getRegionByKey(region.getStartKey());
// When switch leader fails or the region changed its key range,
// it would be necessary to re-split task's key range for new region.
if (!region.getStartKey().equals(cachedRegion.getStartKey())
|| !region.getEndKey().equals(cachedRegion.getEndKey())) {
return false;
}
region = cachedRegion;
String addressStr = regionManager.getStoreById(region.getLeader().getStoreId()).getAddress();
ManagedChannel channel = channelFactory.getChannel(addressStr);
blockingStub = TikvGrpc.newBlockingStub(channel);
asyncStub = TikvGrpc.newStub(channel);
return true;
}
@Override
public void onStoreNotMatch(Metapb.Store store) {
String addressStr = store.getAddress();
ManagedChannel channel = channelFactory.getChannel(addressStr);
blockingStub = TikvGrpc.newBlockingStub(channel);
asyncStub = TikvGrpc.newStub(channel);
if (logger.isDebugEnabled() && region.getLeader().getStoreId() != store.getId()) {
logger.debug(
"store_not_match may occur? "
+ region
+ ", original store = "
+ store.getId()
+ " address = "
+ addressStr);
}
}
}

View File

@ -17,7 +17,7 @@
package org.tikv.common.region;
import static org.tikv.common.codec.KeyUtils.formatBytes;
import static org.tikv.common.codec.KeyUtils.formatBytesUTF8;
import static org.tikv.common.util.KeyRangeUtils.makeRange;
import com.google.common.collect.RangeMap;
@ -27,20 +27,25 @@ import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.log4j.Logger;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.tikv.common.ReadOnlyPDClient;
import org.tikv.common.exception.GrpcException;
import org.tikv.common.exception.TiClientInternalException;
import org.tikv.common.key.Key;
import org.tikv.common.util.BackOffer;
import org.tikv.common.util.ConcreteBackOffer;
import org.tikv.common.util.Pair;
import org.tikv.kvproto.Metapb.Peer;
import org.tikv.kvproto.Metapb.Store;
import org.tikv.kvproto.Metapb.StoreState;
@SuppressWarnings("UnstableApiUsage")
public class RegionManager {
private static final Logger logger = Logger.getLogger(RegionManager.class);
private RegionCache cache;
private static final Logger logger = LoggerFactory.getLogger(RegionManager.class);
// TODO: the region cache logic need rewrite.
// https://github.com/pingcap/tispark/issues/1170
private final RegionCache cache;
// To avoid double retrieval, we used the async version of grpc
// When rpc not returned, instead of call again, it wait for previous one done
@ -48,6 +53,98 @@ public class RegionManager {
this.cache = new RegionCache(pdClient);
}
public TiRegion getRegionByKey(ByteString key) {
return getRegionByKey(key, ConcreteBackOffer.newGetBackOff());
}
public TiRegion getRegionByKey(ByteString key, BackOffer backOffer) {
return cache.getRegionByKey(key, backOffer);
}
@Deprecated
// Do not use GetRegionByID when retrying request.
//
// A,B |_______|_____|
// A |_____________|
// Consider region A, B. After merge of (A, B) -> A, region ID B does not exist.
// This request is unrecoverable.
public TiRegion getRegionById(long regionId) {
return cache.getRegionById(ConcreteBackOffer.newGetBackOff(), regionId);
}
public Pair<TiRegion, Store> getRegionStorePairByKey(ByteString key) {
return getRegionStorePairByKey(key, ConcreteBackOffer.newGetBackOff());
}
public Pair<TiRegion, Store> getRegionStorePairByKey(ByteString key, BackOffer backOffer) {
TiRegion region = cache.getRegionByKey(key, backOffer);
if (region == null) {
throw new TiClientInternalException("Region not exist for key:" + formatBytesUTF8(key));
}
if (!region.isValid()) {
throw new TiClientInternalException("Region invalid: " + region.toString());
}
Peer leader = region.getLeader();
Store store = cache.getStoreById(leader.getStoreId(), backOffer);
if (store == null) {
throw new TiClientInternalException(
"Cannot find valid store on TiKV for region " + region.toString());
}
return Pair.create(region, store);
}
public Store getStoreById(long id) {
return getStoreById(id, ConcreteBackOffer.newGetBackOff());
}
public Store getStoreById(long id, BackOffer backOffer) {
return cache.getStoreById(id, backOffer);
}
public void onRegionStale(long regionId) {
cache.invalidateRegion(regionId);
}
public boolean updateLeader(long regionId, long storeId) {
TiRegion r = cache.regionCache.get(regionId);
if (r != null) {
if (!r.switchPeer(storeId)) {
// failed to switch leader, possibly region is outdated, we need to drop region cache from
// regionCache
logger.warn("Cannot find peer when updating leader (" + regionId + "," + storeId + ")");
// drop region cache using verId
cache.invalidateRegion(regionId);
return false;
}
}
return true;
}
/**
* Clears all cache when a TiKV server does not respond
*
* @param region region
*/
public void onRequestFail(TiRegion region) {
onRequestFail(region.getId(), region.getLeader().getStoreId());
}
public void onRequestFail(long regionId, long storeId) {
cache.invalidateRegion(regionId);
cache.invalidateAllRegionForStore(storeId);
}
public void invalidateStore(long storeId) {
cache.invalidateStore(storeId);
}
public void invalidateRegion(long regionId) {
cache.invalidateRegion(regionId);
}
public static class RegionCache {
private final Map<Long, TiRegion> regionCache;
private final Map<Long, Store> storeCache;
@ -62,16 +159,17 @@ public class RegionManager {
this.pdClient = pdClient;
}
synchronized TiRegion getRegionByKey(ByteString key) {
public synchronized TiRegion getRegionByKey(ByteString key, BackOffer backOffer) {
Long regionId;
regionId = keyToRegionIdCache.get(Key.toRawKey(key));
if (logger.isDebugEnabled()) {
logger.debug(String.format("getRegionByKey key[%s] -> ID[%s]", formatBytes(key), regionId));
logger.debug(
String.format("getRegionByKey key[%s] -> ID[%s]", formatBytesUTF8(key), regionId));
}
if (regionId == null) {
logger.debug("Key not found in keyToRegionIdCache:" + formatBytes(key));
TiRegion region = pdClient.getRegionByKey(ConcreteBackOffer.newGetBackOff(), key);
logger.debug("Key not find in keyToRegionIdCache:" + formatBytesUTF8(key));
TiRegion region = pdClient.getRegionByKey(backOffer, key);
if (!putRegion(region)) {
throw new TiClientInternalException("Invalid Region: " + region.toString());
}
@ -94,13 +192,13 @@ public class RegionManager {
return true;
}
private synchronized TiRegion getRegionById(long regionId) {
private synchronized TiRegion getRegionById(BackOffer backOffer, long regionId) {
TiRegion region = regionCache.get(regionId);
if (logger.isDebugEnabled()) {
logger.debug(String.format("getRegionByKey ID[%s] -> Region[%s]", regionId, region));
}
if (region == null) {
region = pdClient.getRegionByID(ConcreteBackOffer.newGetBackOff(), regionId);
region = pdClient.getRegionByID(backOffer, regionId);
if (!putRegion(region)) {
throw new TiClientInternalException("Invalid Region: " + region.toString());
}
@ -144,18 +242,15 @@ public class RegionManager {
storeCache.remove(storeId);
}
public synchronized Store getStoreById(long id) {
public synchronized Store getStoreById(long id, BackOffer backOffer) {
try {
Store store = storeCache.get(id);
if (store == null) {
store = pdClient.getStore(ConcreteBackOffer.newGetBackOff(), id);
store = pdClient.getStore(backOffer, id);
}
if (store.getState().equals(StoreState.Tombstone)) {
return null;
}
if (logger.isDebugEnabled()) {
logger.debug(String.format("getStoreById ID[%s] -> Store[%s]", id, store));
}
storeCache.put(id, store);
return store;
} catch (Exception e) {
@ -163,77 +258,4 @@ public class RegionManager {
}
}
}
public TiRegion getRegionByKey(ByteString key) {
return cache.getRegionByKey(key);
}
public TiRegion getRegionById(long regionId) {
return cache.getRegionById(regionId);
}
public Pair<TiRegion, Store> getRegionStorePairByKey(ByteString key) {
TiRegion region = cache.getRegionByKey(key);
if (region == null) {
throw new TiClientInternalException("Region not exist for key:" + formatBytes(key));
}
if (!region.isValid()) {
throw new TiClientInternalException("Region invalid: " + region.toString());
}
Peer leader = region.getLeader();
long storeId = leader.getStoreId();
return Pair.create(region, cache.getStoreById(storeId));
}
public Pair<TiRegion, Store> getRegionStorePairByRegionId(long id) {
TiRegion region = cache.getRegionById(id);
if (!region.isValid()) {
throw new TiClientInternalException("Region invalid: " + region.toString());
}
Peer leader = region.getLeader();
long storeId = leader.getStoreId();
return Pair.create(region, cache.getStoreById(storeId));
}
public Store getStoreById(long id) {
return cache.getStoreById(id);
}
public void onRegionStale(long regionId) {
cache.invalidateRegion(regionId);
}
public boolean checkAndDropLeader(long regionId, long storeId) {
TiRegion r = cache.regionCache.get(regionId);
if (r != null) {
TiRegion r2 = r.withNewLeader(storeId);
// drop region cache using verId
cache.invalidateRegion(regionId);
if (r2.getLeader().getStoreId() != storeId) {
// failed to switch leader, possibly region is outdated, we need to drop region cache from
// regionCache
logger.warn("Cannot find peer when updating leader (" + regionId + "," + storeId + ")");
return false;
}
}
return true;
}
/**
* Clears all cache when a TiKV server does not respond
*
* @param region region
*/
public void onRequestFail(TiRegion region) {
cache.invalidateRegion(region.getId());
cache.invalidateAllRegionForStore(region.getLeader().getStoreId());
}
public void invalidateStore(long storeId) {
cache.invalidateStore(storeId);
}
public void invalidateRegion(long regionId) {
cache.invalidateRegion(regionId);
}
}

View File

@ -17,134 +17,147 @@
package org.tikv.common.region;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkNotNull;
import static org.tikv.common.util.BackOffFunction.BackOffFuncType.BoRegionMiss;
import static org.tikv.common.util.BackOffFunction.BackOffFuncType.BoTxnLockFast;
import static org.tikv.common.util.BackOffFunction.BackOffFuncType.*;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.ByteString;
import io.grpc.ManagedChannel;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import java.util.*;
import java.util.function.Supplier;
import org.apache.log4j.Logger;
import org.tikv.common.AbstractGRPCClient;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.tikv.common.PDClient;
import org.tikv.common.TiConfiguration;
import org.tikv.common.exception.KeyException;
import org.tikv.common.exception.RegionException;
import org.tikv.common.exception.TiClientInternalException;
import org.tikv.common.exception.*;
import org.tikv.common.operation.KVErrorHandler;
import org.tikv.common.util.BackOffer;
import org.tikv.common.util.ChannelFactory;
import org.tikv.common.util.Pair;
import org.tikv.kvproto.Kvrpcpb.BatchGetRequest;
import org.tikv.kvproto.Kvrpcpb.BatchGetResponse;
import org.tikv.kvproto.Kvrpcpb.GetRequest;
import org.tikv.kvproto.Kvrpcpb.GetResponse;
import org.tikv.kvproto.Kvrpcpb.KvPair;
import org.tikv.kvproto.Kvrpcpb.RawBatchPutRequest;
import org.tikv.kvproto.Kvrpcpb.RawBatchPutResponse;
import org.tikv.kvproto.Kvrpcpb.RawDeleteRequest;
import org.tikv.kvproto.Kvrpcpb.RawDeleteResponse;
import org.tikv.kvproto.Kvrpcpb.RawGetRequest;
import org.tikv.kvproto.Kvrpcpb.RawGetResponse;
import org.tikv.kvproto.Kvrpcpb.RawPutRequest;
import org.tikv.kvproto.Kvrpcpb.RawPutResponse;
import org.tikv.kvproto.Kvrpcpb.RawScanRequest;
import org.tikv.kvproto.Kvrpcpb.RawScanResponse;
import org.tikv.kvproto.Kvrpcpb.ScanRequest;
import org.tikv.kvproto.Kvrpcpb.ScanResponse;
import org.tikv.kvproto.Kvrpcpb.*;
import org.tikv.kvproto.Metapb.Store;
import org.tikv.kvproto.TikvGrpc;
import org.tikv.kvproto.TikvGrpc.TikvBlockingStub;
import org.tikv.kvproto.TikvGrpc.TikvStub;
import org.tikv.txn.AbstractLockResolverClient;
import org.tikv.txn.Lock;
import org.tikv.txn.LockResolverClient;
import org.tikv.txn.ResolveLockResult;
// RegionStore itself is not thread-safe
public class RegionStoreClient extends AbstractGRPCClient<TikvBlockingStub, TikvStub>
implements RegionErrorReceiver {
// TODO:
// 1. RegionStoreClient will be inaccessible directly.
// 2. All apis of RegionStoreClient would not provide retry aside from callWithRetry,
// if a request needs to be retried because of an un-retryable cause, e.g., keys
// need to be re-split across regions/stores, region info outdated, e.t.c., you
// should retry it in an upper client logic (KVClient, TxnClient, e.t.c.)
private static final Logger logger = Logger.getLogger(RegionStoreClient.class);
private TiRegion region;
private final RegionManager regionManager;
@VisibleForTesting private final LockResolverClient lockResolverClient;
private TikvBlockingStub blockingStub;
private TikvStub asyncStub;
/** Note that RegionStoreClient itself is not thread-safe */
public class RegionStoreClient extends AbstractRegionStoreClient {
private static final Logger logger = LoggerFactory.getLogger(RegionStoreClient.class);
@VisibleForTesting public final AbstractLockResolverClient lockResolverClient;
/** startTS -> List(locks) */
private final Map<Long, Set<Long>> resolvedLocks = new HashMap<>();
public TiRegion getRegion() {
return region;
private RegionStoreClient(
TiConfiguration conf,
TiRegion region,
Store store,
ChannelFactory channelFactory,
TikvBlockingStub blockingStub,
TikvStub asyncStub,
RegionManager regionManager,
PDClient pdClient,
RegionStoreClient.RegionStoreClientBuilder clientBuilder) {
super(conf, region, channelFactory, blockingStub, asyncStub, regionManager);
this.lockResolverClient =
AbstractLockResolverClient.getInstance(
store,
conf,
region,
this.blockingStub,
this.asyncStub,
channelFactory,
regionManager,
pdClient,
clientBuilder);
}
// APIs for KV Scan/Put/Get/Delete
public ByteString get(BackOffer backOffer, ByteString key, long version) {
while (true) {
Supplier<GetRequest> factory =
() ->
GetRequest.newBuilder()
.setContext(region.getContext())
.setKey(key)
.setVersion(version)
.build();
KVErrorHandler<GetResponse> handler =
new KVErrorHandler<>(
regionManager,
this,
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null);
GetResponse resp = callWithRetry(backOffer, TikvGrpc.METHOD_KV_GET, factory, handler);
if (getHelper(backOffer, resp)) {
return resp.getValue();
}
// we should refresh region
region = regionManager.getRegionByKey(key);
}
}
private boolean getHelper(BackOffer backOffer, GetResponse resp) {
if (resp == null) {
this.regionManager.onRequestFail(region);
throw new TiClientInternalException("GetResponse failed without a cause");
}
if (resp.hasRegionError()) {
backOffer.doBackOff(BoRegionMiss, new RegionException(resp.getRegionError()));
return false;
}
if (resp.hasError()) {
if (resp.getError().hasLocked()) {
Lock lock = new Lock(resp.getError().getLocked());
boolean ok =
lockResolverClient.resolveLocks(backOffer, new ArrayList<>(Arrays.asList(lock)));
if (!ok) {
// if not resolve all locks, we wait and retry
backOffer.doBackOff(
BoTxnLockFast, new KeyException((resp.getError().getLocked().toString())));
}
return false;
} else {
// retry or abort
// this should trigger Spark to retry the txn
throw new KeyException(resp.getError());
}
public synchronized boolean addResolvedLocks(Long version, List<Long> locks) {
Set<Long> oldList = resolvedLocks.get(version);
if (oldList != null) {
oldList.addAll(locks);
} else {
resolvedLocks.put(version, new HashSet<>(locks));
}
return true;
}
// TODO: batch get should consider key range split
public synchronized Set<Long> getResolvedLocks(Long version) {
return resolvedLocks.getOrDefault(version, java.util.Collections.emptySet());
}
/**
* Fetch a value according to a key
*
* @param backOffer backOffer
* @param key key to fetch
* @param version key version
* @return value
* @throws TiClientInternalException TiSpark Client exception, unexpected
* @throws KeyException Key may be locked
*/
public ByteString get(BackOffer backOffer, ByteString key, long version)
throws TiClientInternalException, KeyException {
boolean forWrite = false;
Supplier<GetRequest> factory =
() ->
GetRequest.newBuilder()
.setContext(region.getContext(getResolvedLocks(version)))
.setKey(key)
.setVersion(version)
.build();
KVErrorHandler<GetResponse> handler =
new KVErrorHandler<>(
regionManager,
this,
lockResolverClient,
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null,
resp -> resp.hasError() ? resp.getError() : null,
resolveLockResult -> addResolvedLocks(version, resolveLockResult.getResolvedLocks()),
version,
forWrite);
GetResponse resp = callWithRetry(backOffer, TikvGrpc.getKvGetMethod(), factory, handler);
handleGetResponse(resp);
return resp.getValue();
}
/**
* @param resp GetResponse
* @throws TiClientInternalException TiSpark Client exception, unexpected
* @throws KeyException Key may be locked
*/
private void handleGetResponse(GetResponse resp) throws TiClientInternalException, KeyException {
if (resp == null) {
this.regionManager.onRequestFail(region);
throw new TiClientInternalException("GetResponse failed without a cause");
}
if (resp.hasRegionError()) {
throw new RegionException(resp.getRegionError());
}
if (resp.hasError()) {
throw new KeyException(resp.getError());
}
}
public List<KvPair> batchGet(BackOffer backOffer, Iterable<ByteString> keys, long version) {
boolean forWrite = false;
Supplier<BatchGetRequest> request =
() ->
BatchGetRequest.newBuilder()
.setContext(region.getContext())
.setContext(region.getContext(getResolvedLocks(version)))
.addAllKeys(keys)
.setVersion(version)
.build();
@ -152,15 +165,28 @@ public class RegionStoreClient extends AbstractGRPCClient<TikvBlockingStub, Tikv
new KVErrorHandler<>(
regionManager,
this,
lockResolverClient,
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null);
resp -> resp.hasRegionError() ? resp.getRegionError() : null,
resp -> null,
resolveLockResult -> addResolvedLocks(version, resolveLockResult.getResolvedLocks()),
version,
forWrite);
BatchGetResponse resp =
callWithRetry(backOffer, TikvGrpc.METHOD_KV_BATCH_GET, request, handler);
return batchGetHelper(resp, backOffer);
callWithRetry(backOffer, TikvGrpc.getKvBatchGetMethod(), request, handler);
return handleBatchGetResponse(backOffer, resp, version);
}
// TODO: deal with resolve locks and region errors
private List<KvPair> batchGetHelper(BatchGetResponse resp, BackOffer bo) {
private List<KvPair> handleBatchGetResponse(
BackOffer backOffer, BatchGetResponse resp, long version) {
boolean forWrite = false;
if (resp == null) {
this.regionManager.onRequestFail(region);
throw new TiClientInternalException("BatchGetResponse failed without a cause");
}
if (resp.hasRegionError()) {
throw new RegionException(resp.getRegionError());
}
List<Lock> locks = new ArrayList<>();
for (KvPair pair : resp.getPairsList()) {
@ -175,88 +201,309 @@ public class RegionStoreClient extends AbstractGRPCClient<TikvBlockingStub, Tikv
}
if (!locks.isEmpty()) {
boolean ok = lockResolverClient.resolveLocks(bo, locks);
if (!ok) {
// if not resolve all locks, we wait and retry
bo.doBackOff(BoTxnLockFast, new KeyException((resp.getPairsList().get(0).getError())));
}
// TODO: we should retry
// fix me
ResolveLockResult resolveLockResult =
lockResolverClient.resolveLocks(backOffer, version, locks, forWrite);
addResolvedLocks(version, resolveLockResult.getResolvedLocks());
// resolveLocks already retried, just throw error to upper logic.
throw new TiKVException("locks not resolved, retry");
} else {
return resp.getPairsList();
}
if (resp.hasRegionError()) {
// TODO, we should redo the split and redo the batchGet
throw new RegionException(resp.getRegionError());
}
return resp.getPairsList();
}
public List<KvPair> scan(
BackOffer backOffer, ByteString startKey, long version, boolean keyOnly) {
Supplier<ScanRequest> request =
() ->
ScanRequest.newBuilder()
.setContext(region.getContext())
.setStartKey(startKey)
.setVersion(version)
.setKeyOnly(keyOnly)
.setLimit(getConf().getScanBatchSize())
.build();
boolean forWrite = false;
while (true) {
// we should refresh region
region = regionManager.getRegionByKey(startKey);
KVErrorHandler<ScanResponse> handler =
new KVErrorHandler<>(
regionManager,
this,
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null);
ScanResponse resp = callWithRetry(backOffer, TikvGrpc.METHOD_KV_SCAN, request, handler);
return scanHelper(resp, backOffer);
Supplier<ScanRequest> request =
() ->
ScanRequest.newBuilder()
.setContext(region.getContext(getResolvedLocks(version)))
.setStartKey(startKey)
.setVersion(version)
.setKeyOnly(keyOnly)
.setLimit(getConf().getScanBatchSize())
.build();
KVErrorHandler<ScanResponse> handler =
new KVErrorHandler<>(
regionManager,
this,
lockResolverClient,
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null,
resp -> null,
resolveLockResult -> addResolvedLocks(version, resolveLockResult.getResolvedLocks()),
version,
forWrite);
ScanResponse resp = callWithRetry(backOffer, TikvGrpc.getKvScanMethod(), request, handler);
if (isScanSuccess(backOffer, resp)) {
return doScan(resp);
}
}
}
// TODO: remove helper and change to while style
// needs to be fixed as batchGet
// which we shoule retry not throw
// exception
private List<KvPair> scanHelper(ScanResponse resp, BackOffer bo) {
private boolean isScanSuccess(BackOffer backOffer, ScanResponse resp) {
if (resp == null) {
this.regionManager.onRequestFail(region);
throw new TiClientInternalException("ScanResponse failed without a cause");
}
List<Lock> locks = new ArrayList<>();
for (KvPair pair : resp.getPairsList()) {
if (pair.hasError()) {
if (pair.getError().hasLocked()) {
Lock lock = new Lock(pair.getError().getLocked());
locks.add(lock);
} else {
throw new KeyException(pair.getError());
}
}
}
if (!locks.isEmpty()) {
boolean ok = lockResolverClient.resolveLocks(bo, locks);
if (!ok) {
// if not resolve all locks, we wait and retry
bo.doBackOff(BoTxnLockFast, new KeyException((resp.getPairsList().get(0).getError())));
}
// TODO: we should retry
// fix me
}
if (resp.hasRegionError()) {
throw new RegionException(resp.getRegionError());
backOffer.doBackOff(BoRegionMiss, new RegionException(resp.getRegionError()));
return false;
}
return resp.getPairsList();
return true;
}
// TODO: resolve locks after scan
private List<KvPair> doScan(ScanResponse resp) {
// Check if kvPair contains error, it should be a Lock if hasError is true.
List<KvPair> kvPairs = resp.getPairsList();
List<KvPair> newKvPairs = new ArrayList<>();
for (KvPair kvPair : kvPairs) {
if (kvPair.hasError()) {
Lock lock = AbstractLockResolverClient.extractLockFromKeyErr(kvPair.getError());
newKvPairs.add(
KvPair.newBuilder()
.setError(kvPair.getError())
.setValue(kvPair.getValue())
.setKey(lock.getKey())
.build());
} else {
newKvPairs.add(kvPair);
}
}
return Collections.unmodifiableList(newKvPairs);
}
public List<KvPair> scan(BackOffer backOffer, ByteString startKey, long version) {
return scan(backOffer, startKey, version, false);
}
/**
* Prewrite batch keys
*
* @param backOffer backOffer
* @param primary primary lock of keys
* @param mutations batch key-values as mutations
* @param startTs startTs of prewrite
* @param lockTTL lock ttl
* @throws TiClientInternalException TiSpark Client exception, unexpected
* @throws KeyException Key may be locked
* @throws RegionException region error occurs
*/
public void prewrite(
BackOffer backOffer,
ByteString primary,
Iterable<Mutation> mutations,
long startTs,
long lockTTL)
throws TiClientInternalException, KeyException, RegionException {
this.prewrite(backOffer, primary, mutations, startTs, lockTTL, false);
}
/**
* Prewrite batch keys
*
* @param skipConstraintCheck whether to skip constraint check
*/
public void prewrite(
BackOffer bo,
ByteString primaryLock,
Iterable<Mutation> mutations,
long startTs,
long ttl,
boolean skipConstraintCheck)
throws TiClientInternalException, KeyException, RegionException {
boolean forWrite = true;
while (true) {
Supplier<PrewriteRequest> factory =
() ->
PrewriteRequest.newBuilder()
.setContext(region.getContext())
.setStartVersion(startTs)
.setPrimaryLock(primaryLock)
.addAllMutations(mutations)
.setLockTtl(ttl)
.setSkipConstraintCheck(skipConstraintCheck)
.setMinCommitTs(startTs)
.setTxnSize(16)
.build();
KVErrorHandler<PrewriteResponse> handler =
new KVErrorHandler<>(
regionManager,
this,
lockResolverClient,
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null,
resp -> null,
resolveLockResult -> null,
startTs,
forWrite);
PrewriteResponse resp = callWithRetry(bo, TikvGrpc.getKvPrewriteMethod(), factory, handler);
if (isPrewriteSuccess(bo, resp, startTs)) {
return;
}
}
}
/**
* @param backOffer backOffer
* @param resp response
* @return Return true means the rpc call success. Return false means the rpc call fail,
* RegionStoreClient should retry. Throw an Exception means the rpc call fail,
* RegionStoreClient cannot handle this kind of error
* @throws TiClientInternalException
* @throws RegionException
* @throws KeyException
*/
private boolean isPrewriteSuccess(BackOffer backOffer, PrewriteResponse resp, long startTs)
throws TiClientInternalException, KeyException, RegionException {
boolean forWrite = true;
if (resp == null) {
this.regionManager.onRequestFail(region);
throw new TiClientInternalException("Prewrite Response failed without a cause");
}
if (resp.hasRegionError()) {
throw new RegionException(resp.getRegionError());
}
boolean isSuccess = true;
List<Lock> locks = new ArrayList<>();
for (KeyError err : resp.getErrorsList()) {
if (err.hasLocked()) {
isSuccess = false;
Lock lock = new Lock(err.getLocked());
locks.add(lock);
} else {
throw new KeyException(err.toString());
}
}
if (isSuccess) {
return true;
}
ResolveLockResult resolveLockResult =
lockResolverClient.resolveLocks(backOffer, startTs, locks, forWrite);
addResolvedLocks(startTs, resolveLockResult.getResolvedLocks());
long msBeforeExpired = resolveLockResult.getMsBeforeTxnExpired();
if (msBeforeExpired > 0) {
backOffer.doBackOffWithMaxSleep(
BoTxnLock, msBeforeExpired, new KeyException(resp.getErrorsList().get(0)));
}
return false;
}
/** TXN Heart Beat: update primary key ttl */
public void txnHeartBeat(BackOffer bo, ByteString primaryLock, long startTs, long ttl) {
boolean forWrite = false;
while (true) {
Supplier<TxnHeartBeatRequest> factory =
() ->
TxnHeartBeatRequest.newBuilder()
.setContext(region.getContext())
.setStartVersion(startTs)
.setPrimaryLock(primaryLock)
.setAdviseLockTtl(ttl)
.build();
KVErrorHandler<TxnHeartBeatResponse> handler =
new KVErrorHandler<>(
regionManager,
this,
lockResolverClient,
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null,
resp -> resp.hasError() ? resp.getError() : null,
resolveLockResult -> null,
startTs,
forWrite);
TxnHeartBeatResponse resp =
callWithRetry(bo, TikvGrpc.getKvTxnHeartBeatMethod(), factory, handler);
if (isTxnHeartBeatSuccess(resp)) {
return;
}
}
}
private boolean isTxnHeartBeatSuccess(TxnHeartBeatResponse resp)
throws TiClientInternalException, RegionException {
if (resp == null) {
this.regionManager.onRequestFail(region);
throw new TiClientInternalException("TxnHeartBeat Response failed without a cause");
}
if (resp.hasRegionError()) {
throw new RegionException(resp.getRegionError());
}
if (resp.hasError()) {
throw new TiClientInternalException("TxnHeartBeat fail, " + resp.getError().getAbort());
}
return true;
}
/**
* Commit batch keys
*
* @param backOffer backOffer
* @param keys keys to commit
* @param startTs start version
* @param commitTs commit version
*/
public void commit(BackOffer backOffer, Iterable<ByteString> keys, long startTs, long commitTs)
throws KeyException {
boolean forWrite = true;
Supplier<CommitRequest> factory =
() ->
CommitRequest.newBuilder()
.setStartVersion(startTs)
.setCommitVersion(commitTs)
.addAllKeys(keys)
.setContext(region.getContext())
.build();
KVErrorHandler<CommitResponse> handler =
new KVErrorHandler<>(
regionManager,
this,
lockResolverClient,
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null,
resp -> resp.hasError() ? resp.getError() : null,
resolveLockResult -> null,
startTs,
forWrite);
CommitResponse resp = callWithRetry(backOffer, TikvGrpc.getKvCommitMethod(), factory, handler);
handleCommitResponse(resp);
}
/**
* @param resp CommitResponse
* @throws TiClientInternalException
* @throws RegionException
* @throws KeyException
*/
private void handleCommitResponse(CommitResponse resp)
throws TiClientInternalException, RegionException, KeyException {
if (resp == null) {
this.regionManager.onRequestFail(region);
throw new TiClientInternalException("CommitResponse failed without a cause");
}
if (resp.hasRegionError()) {
// bo.doBackOff(BoRegionMiss, new RegionException(resp.getRegionError()));
// return false;
// Caller method should restart commit
throw new RegionException(resp.getRegionError());
}
// If we find locks, we first resolve and let its caller retry.
if (resp.hasError()) {
throw new KeyException(resp.getError());
}
}
// APIs for Raw Scan/Put/Get/Delete
public ByteString rawGet(BackOffer backOffer, ByteString key) {
@ -268,7 +515,7 @@ public class RegionStoreClient extends AbstractGRPCClient<TikvBlockingStub, Tikv
this,
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null);
RawGetResponse resp = callWithRetry(backOffer, TikvGrpc.METHOD_RAW_GET, factory, handler);
RawGetResponse resp = callWithRetry(backOffer, TikvGrpc.getRawGetMethod(), factory, handler);
return rawGetHelper(resp);
}
@ -297,7 +544,8 @@ public class RegionStoreClient extends AbstractGRPCClient<TikvBlockingStub, Tikv
this,
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null);
RawDeleteResponse resp = callWithRetry(backOffer, TikvGrpc.METHOD_RAW_DELETE, factory, handler);
RawDeleteResponse resp =
callWithRetry(backOffer, TikvGrpc.getRawDeleteMethod(), factory, handler);
rawDeleteHelper(resp, region);
}
@ -330,7 +578,7 @@ public class RegionStoreClient extends AbstractGRPCClient<TikvBlockingStub, Tikv
this,
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null);
RawPutResponse resp = callWithRetry(backOffer, TikvGrpc.METHOD_RAW_PUT, factory, handler);
RawPutResponse resp = callWithRetry(backOffer, TikvGrpc.getRawPutMethod(), factory, handler);
rawPutHelper(resp);
}
@ -365,7 +613,7 @@ public class RegionStoreClient extends AbstractGRPCClient<TikvBlockingStub, Tikv
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null);
RawBatchPutResponse resp =
callWithRetry(backOffer, TikvGrpc.METHOD_RAW_BATCH_PUT, factory, handler);
callWithRetry(backOffer, TikvGrpc.getRawBatchPutMethod(), factory, handler);
handleRawBatchPut(resp);
}
@ -404,7 +652,7 @@ public class RegionStoreClient extends AbstractGRPCClient<TikvBlockingStub, Tikv
this,
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null);
RawScanResponse resp = callWithRetry(backOffer, TikvGrpc.METHOD_RAW_SCAN, factory, handler);
RawScanResponse resp = callWithRetry(backOffer, TikvGrpc.getRawScanMethod(), factory, handler);
return rawScanHelper(resp);
}
@ -427,22 +675,45 @@ public class RegionStoreClient extends AbstractGRPCClient<TikvBlockingStub, Tikv
return resp.getKvsList();
}
public enum RequestTypes {
REQ_TYPE_SELECT(101),
REQ_TYPE_INDEX(102),
REQ_TYPE_DAG(103),
REQ_TYPE_ANALYZE(104),
BATCH_ROW_COUNT(64);
private final int value;
RequestTypes(int value) {
this.value = value;
}
public int getValue() {
return value;
}
}
public static class RegionStoreClientBuilder {
private final TiConfiguration conf;
private final ChannelFactory channelFactory;
private final RegionManager regionManager;
private final PDClient pdClient;
public RegionStoreClientBuilder(
TiConfiguration conf, ChannelFactory channelFactory, RegionManager regionManager) {
TiConfiguration conf,
ChannelFactory channelFactory,
RegionManager regionManager,
PDClient pdClient) {
Objects.requireNonNull(conf, "conf is null");
Objects.requireNonNull(channelFactory, "channelFactory is null");
Objects.requireNonNull(regionManager, "regionManager is null");
this.conf = conf;
this.channelFactory = channelFactory;
this.regionManager = regionManager;
this.pdClient = pdClient;
}
public RegionStoreClient build(TiRegion region, Store store) {
public RegionStoreClient build(TiRegion region, Store store) throws GrpcException {
Objects.requireNonNull(region, "region is null");
Objects.requireNonNull(store, "store is null");
@ -456,7 +727,15 @@ public class RegionStoreClient extends AbstractGRPCClient<TikvBlockingStub, Tikv
TikvStub asyncStub = TikvGrpc.newStub(channel);
return new RegionStoreClient(
conf, region, channelFactory, blockingStub, asyncStub, regionManager);
conf,
region,
store,
channelFactory,
blockingStub,
asyncStub,
regionManager,
pdClient,
this);
}
public RegionStoreClient build(ByteString key) {
@ -464,7 +743,7 @@ public class RegionStoreClient extends AbstractGRPCClient<TikvBlockingStub, Tikv
return build(pair.first, pair.second);
}
public RegionStoreClient build(TiRegion region) {
public RegionStoreClient build(TiRegion region) throws GrpcException {
Store store = regionManager.getStoreById(region.getLeader().getStoreId());
return build(region, store);
}
@ -473,80 +752,4 @@ public class RegionStoreClient extends AbstractGRPCClient<TikvBlockingStub, Tikv
return regionManager;
}
}
private RegionStoreClient(
TiConfiguration conf,
TiRegion region,
ChannelFactory channelFactory,
TikvBlockingStub blockingStub,
TikvStub asyncStub,
RegionManager regionManager) {
super(conf, channelFactory);
checkNotNull(region, "Region is empty");
checkNotNull(region.getLeader(), "Leader Peer is null");
checkArgument(region.getLeader() != null, "Leader Peer is null");
this.regionManager = regionManager;
this.region = region;
this.blockingStub = blockingStub;
this.asyncStub = asyncStub;
this.lockResolverClient =
new LockResolverClient(
conf, this.blockingStub, this.asyncStub, channelFactory, regionManager);
}
@Override
protected TikvBlockingStub getBlockingStub() {
return blockingStub.withDeadlineAfter(getConf().getTimeout(), getConf().getTimeoutUnit());
}
@Override
protected TikvStub getAsyncStub() {
return asyncStub.withDeadlineAfter(getConf().getTimeout(), getConf().getTimeoutUnit());
}
@Override
public void close() throws Exception {}
/**
* onNotLeader deals with NotLeaderError and returns whether re-splitting key range is needed
*
* @param newStore the new store presented by NotLeader Error
* @return false when re-split is needed.
*/
@Override
public boolean onNotLeader(Store newStore) {
if (logger.isDebugEnabled()) {
logger.debug(region + ", new leader = " + newStore.getId());
}
TiRegion cachedRegion = regionManager.getRegionById(region.getId());
// When switch leader fails or the region changed its key range,
// it would be necessary to re-split task's key range for new region.
if (!region.getStartKey().equals(cachedRegion.getStartKey())
|| !region.getEndKey().equals(cachedRegion.getEndKey())) {
return false;
}
region = cachedRegion;
String addressStr = regionManager.getStoreById(region.getLeader().getStoreId()).getAddress();
ManagedChannel channel = channelFactory.getChannel(addressStr);
blockingStub = TikvGrpc.newBlockingStub(channel);
asyncStub = TikvGrpc.newStub(channel);
return true;
}
@Override
public void onStoreNotMatch(Store store) {
String addressStr = store.getAddress();
ManagedChannel channel = channelFactory.getChannel(addressStr);
blockingStub = TikvGrpc.newBlockingStub(channel);
asyncStub = TikvGrpc.newStub(channel);
if (logger.isDebugEnabled() && region.getLeader().getStoreId() != store.getId()) {
logger.debug(
"store_not_match may occur? "
+ region
+ ", original store = "
+ store.getId()
+ " address = "
+ addressStr);
}
}
}

View File

@ -21,12 +21,15 @@ import com.google.protobuf.ByteString;
import java.io.Serializable;
import java.util.List;
import java.util.Objects;
import java.util.Set;
import org.tikv.common.TiConfiguration.KVMode;
import org.tikv.common.codec.Codec.BytesCodec;
import org.tikv.common.codec.CodecDataInput;
import org.tikv.common.codec.KeyUtils;
import org.tikv.common.exception.TiClientInternalException;
import org.tikv.common.key.Key;
import org.tikv.common.util.FastByteComparisons;
import org.tikv.common.util.KeyRangeUtils;
import org.tikv.kvproto.Kvrpcpb;
import org.tikv.kvproto.Kvrpcpb.IsolationLevel;
import org.tikv.kvproto.Metapb;
@ -35,10 +38,9 @@ import org.tikv.kvproto.Metapb.Region;
public class TiRegion implements Serializable {
private final Region meta;
private final Peer peer;
private final IsolationLevel isolationLevel;
private final Kvrpcpb.CommandPri commandPri;
private Kvrpcpb.Context cachedContext;
private Peer peer;
public TiRegion(
Region meta,
@ -52,6 +54,7 @@ public class TiRegion implements Serializable {
if (meta.getPeersCount() == 0) {
throw new TiClientInternalException("Empty peer list for region " + meta.getId());
}
// region's first peer is leader.
this.peer = meta.getPeers(0);
} else {
this.peer = peer;
@ -60,18 +63,6 @@ public class TiRegion implements Serializable {
this.commandPri = commandPri;
}
private TiRegion(
Region meta, Peer peer, IsolationLevel isolationLevel, Kvrpcpb.CommandPri commandPri) {
this.meta = meta;
this.peer = peer;
this.isolationLevel = isolationLevel;
this.commandPri = commandPri;
}
private TiRegion withNewLeader(Peer p) {
return new TiRegion(this.meta, p, this.isolationLevel, this.commandPri);
}
private Region decodeRegion(Region region, boolean isRawRegion) {
Region.Builder builder =
Region.newBuilder()
@ -108,35 +99,29 @@ public class TiRegion implements Serializable {
return meta.getStartKey();
}
public boolean contains(Key key) {
return KeyRangeUtils.makeRange(this.getStartKey(), this.getEndKey()).contains(key);
}
public ByteString getEndKey() {
return meta.getEndKey();
}
public Kvrpcpb.Context getContext() {
if (cachedContext == null) {
Kvrpcpb.Context.Builder builder = Kvrpcpb.Context.newBuilder();
builder.setIsolationLevel(this.isolationLevel);
builder.setPriority(this.commandPri);
builder
.setRegionId(this.meta.getId())
.setPeer(this.peer)
.setRegionEpoch(this.meta.getRegionEpoch());
cachedContext = builder.build();
return cachedContext;
}
return cachedContext;
public Key getRowEndKey() {
return Key.toRawKey(getEndKey());
}
public class RegionVerID {
public final long id;
public final long confVer;
public final long ver;
public Kvrpcpb.Context getContext() {
return getContext(java.util.Collections.emptySet());
}
public RegionVerID(long id, long confVer, long ver) {
this.id = id;
this.confVer = confVer;
this.ver = ver;
}
public Kvrpcpb.Context getContext(Set<Long> resolvedLocks) {
Kvrpcpb.Context.Builder builder = Kvrpcpb.Context.newBuilder();
builder.setIsolationLevel(this.isolationLevel);
builder.setPriority(this.commandPri);
builder.setRegionId(meta.getId()).setPeer(this.peer).setRegionEpoch(this.meta.getRegionEpoch());
builder.addAllResolvedLocks(resolvedLocks);
return builder.build();
}
// getVerID returns the Region's RegionVerID.
@ -150,36 +135,43 @@ public class TiRegion implements Serializable {
* storeID.
*
* @param leaderStoreID is leader peer id.
* @return a copy of current region with new leader store id
* @return false if no peers matches the store id.
*/
public TiRegion withNewLeader(long leaderStoreID) {
boolean switchPeer(long leaderStoreID) {
List<Peer> peers = meta.getPeersList();
for (Peer p : peers) {
if (p.getStoreId() == leaderStoreID) {
return withNewLeader(p);
this.peer = p;
return true;
}
}
return this;
return false;
}
public boolean isMoreThan(ByteString key) {
return FastByteComparisons.compareTo(
meta.getStartKey().toByteArray(),
0,
meta.getStartKey().size(),
key.toByteArray(),
0,
key.size())
> 0;
}
public boolean isLessThan(ByteString key) {
return FastByteComparisons.compareTo(
meta.getEndKey().toByteArray(),
0,
meta.getEndKey().size(),
key.toByteArray(),
0,
key.size())
<= 0;
}
public boolean contains(ByteString key) {
return (FastByteComparisons.compareTo(
meta.getStartKey().toByteArray(),
0,
meta.getStartKey().size(),
key.toByteArray(),
0,
key.size())
<= 0)
&& (meta.getEndKey().isEmpty()
|| FastByteComparisons.compareTo(
meta.getEndKey().toByteArray(),
0,
meta.getEndKey().size(),
key.toByteArray(),
0,
key.size())
> 0);
return !isMoreThan(key) && !isLessThan(key);
}
public boolean isValid() {
@ -194,6 +186,24 @@ public class TiRegion implements Serializable {
return meta;
}
@Override
public boolean equals(final Object another) {
if (!(another instanceof TiRegion)) {
return false;
}
TiRegion anotherRegion = ((TiRegion) another);
return anotherRegion.meta.equals(this.meta)
&& anotherRegion.peer.equals(this.peer)
&& anotherRegion.commandPri.equals(this.commandPri)
&& anotherRegion.isolationLevel.equals(this.isolationLevel);
}
@Override
public int hashCode() {
return Objects.hash(meta, peer, isolationLevel, commandPri);
}
@Override
public String toString() {
return String.format(
"{Region[%d] ConfVer[%d] Version[%d] Store[%d] KeyRange[%s]:[%s]}",
@ -201,7 +211,40 @@ public class TiRegion implements Serializable {
getRegionEpoch().getConfVer(),
getRegionEpoch().getVersion(),
getLeader().getStoreId(),
KeyUtils.formatBytes(getStartKey()),
KeyUtils.formatBytes(getEndKey()));
KeyUtils.formatBytesUTF8(getStartKey()),
KeyUtils.formatBytesUTF8(getEndKey()));
}
public class RegionVerID {
final long id;
final long confVer;
final long ver;
RegionVerID(long id, long confVer, long ver) {
this.id = id;
this.confVer = confVer;
this.ver = ver;
}
@Override
public boolean equals(Object other) {
if (this == other) {
return true;
}
if (!(other instanceof RegionVerID)) {
return false;
}
RegionVerID that = (RegionVerID) other;
return id == that.id && confVer == that.confVer && ver == that.ver;
}
@Override
public int hashCode() {
int hash = Long.hashCode(id);
hash = hash * 31 + Long.hashCode(confVer);
hash = hash * 31 + Long.hashCode(ver);
return hash;
}
}
}

View File

@ -7,14 +7,9 @@ public class BackOffFunction {
private final int base;
private final int cap;
private final BackOffer.BackOffStrategy strategy;
private int lastSleep;
private long lastSleep;
private int attempts;
public static BackOffFunction create(int base, int cap, BackOffer.BackOffStrategy strategy) {
return new BackOffFunction(base, cap, strategy);
}
private BackOffFunction(int base, int cap, BackOffer.BackOffStrategy strategy) {
this.base = base;
this.cap = cap;
@ -22,39 +17,36 @@ public class BackOffFunction {
lastSleep = base;
}
public enum BackOffFuncType {
BoTiKVRPC,
BoTxnLock,
BoTxnLockFast,
BoPDRPC,
BoRegionMiss,
BoUpdateLeader,
BoServerBusy,
BoStoreNotMatch
public static BackOffFunction create(int base, int cap, BackOffer.BackOffStrategy strategy) {
return new BackOffFunction(base, cap, strategy);
}
/**
* Do back off in exponential with optional jitters according to different back off strategies.
* See http://www.awsarchitectureblog.com/2015/03/backoff.html
*/
public int doBackOff() {
int sleep = 0;
int v = expo(base, cap, attempts);
long doBackOff(long maxSleepMs) {
long sleep = 0;
long v = expo(base, cap, attempts);
switch (strategy) {
case NoJitter:
sleep = v;
break;
case FullJitter:
sleep = ThreadLocalRandom.current().nextInt(v);
sleep = ThreadLocalRandom.current().nextLong(v);
break;
case EqualJitter:
sleep = v / 2 + ThreadLocalRandom.current().nextInt(v / 2);
sleep = v / 2 + ThreadLocalRandom.current().nextLong(v / 2);
break;
case DecorrJitter:
sleep = Math.min(cap, base + ThreadLocalRandom.current().nextInt(lastSleep * 3 - base));
sleep = Math.min(cap, base + ThreadLocalRandom.current().nextLong(lastSleep * 3 - base));
break;
}
if (maxSleepMs > 0 && sleep > maxSleepMs) {
sleep = maxSleepMs;
}
try {
Thread.sleep(sleep);
} catch (InterruptedException e) {
@ -68,4 +60,15 @@ public class BackOffFunction {
private int expo(int base, int cap, int n) {
return (int) Math.min(cap, base * Math.pow(2.0d, n));
}
public enum BackOffFuncType {
BoTiKVRPC,
BoTxnLock,
BoTxnLockFast,
BoPDRPC,
BoRegionMiss,
BoUpdateLeader,
BoServerBusy,
BoTxnNotFound
}
}

View File

@ -18,6 +18,37 @@
package org.tikv.common.util;
public interface BackOffer {
// Back off types.
int seconds = 1000;
int COP_BUILD_TASK_MAX_BACKOFF = 5 * seconds;
int TSO_MAX_BACKOFF = 5 * seconds;
int SCANNER_NEXT_MAX_BACKOFF = 40 * seconds;
int BATCH_GET_MAX_BACKOFF = 40 * seconds;
int COP_NEXT_MAX_BACKOFF = 40 * seconds;
int GET_MAX_BACKOFF = 40 * seconds;
int PREWRITE_MAX_BACKOFF = 20 * seconds;
int CLEANUP_MAX_BACKOFF = 20 * seconds;
int GC_ONE_REGION_MAX_BACKOFF = 20 * seconds;
int GC_RESOLVE_LOCK_MAX_BACKOFF = 100 * seconds;
int GC_DELETE_RANGE_MAX_BACKOFF = 100 * seconds;
int RAWKV_MAX_BACKOFF = 40 * seconds;
int SPLIT_REGION_BACKOFF = 20 * seconds;
int BATCH_COMMIT_BACKOFF = 10 * seconds;
int PD_INFO_BACKOFF = 5 * seconds;
/**
* doBackOff sleeps a while base on the BackOffType and records the error message. Will stop until
* max back off time exceeded and throw an exception to the caller.
*/
void doBackOff(BackOffFunction.BackOffFuncType funcType, Exception err);
/**
* BackoffWithMaxSleep sleeps a while base on the backoffType and records the error message and
* never sleep more than maxSleepMs for each sleep.
*/
void doBackOffWithMaxSleep(
BackOffFunction.BackOffFuncType funcType, long maxSleepMs, Exception err);
// Back off strategies
enum BackOffStrategy {
// NoJitter makes the backoff sequence strict exponential.
@ -29,25 +60,4 @@ public interface BackOffer {
// DecorrJitter increases the maximum jitter based on the last random value.
DecorrJitter
}
// Back off types.
int copBuildTaskMaxBackoff = 5000;
int tsoMaxBackoff = 5000;
int scannerNextMaxBackoff = 40000;
int batchGetMaxBackoff = 40000;
int copNextMaxBackoff = 40000;
int getMaxBackoff = 40000;
int prewriteMaxBackoff = 20000;
int cleanupMaxBackoff = 20000;
int GcOneRegionMaxBackoff = 20000;
int GcResolveLockMaxBackoff = 100000;
int GcDeleteRangeMaxBackoff = 100000;
int rawkvMaxBackoff = 40000;
int splitRegionBackoff = 20000;
/**
* doBackOff sleeps a while base on the BackOffType and records the error message. Will stop until
* max back off time exceeded and throw an exception to the caller.
*/
void doBackOff(BackOffFunction.BackOffFuncType funcTypes, Exception err);
}

View File

@ -22,47 +22,16 @@ import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.log4j.Logger;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.tikv.common.exception.GrpcException;
public class ConcreteBackOffer implements BackOffer {
private static final Logger logger = LoggerFactory.getLogger(ConcreteBackOffer.class);
private final int maxSleep;
private int totalSleep;
private final Map<BackOffFunction.BackOffFuncType, BackOffFunction> backOffFunctionMap;
private final List<Exception> errors;
private static final Logger logger = Logger.getLogger(ConcreteBackOffer.class);
public static ConcreteBackOffer newCustomBackOff(int maxSleep) {
return new ConcreteBackOffer(maxSleep);
}
public static ConcreteBackOffer newScannerNextMaxBackOff() {
return new ConcreteBackOffer(scannerNextMaxBackoff);
}
public static ConcreteBackOffer newBatchGetMaxBackOff() {
return new ConcreteBackOffer(batchGetMaxBackoff);
}
public static ConcreteBackOffer newCopNextMaxBackOff() {
return new ConcreteBackOffer(copNextMaxBackoff);
}
public static ConcreteBackOffer newGetBackOff() {
return new ConcreteBackOffer(getMaxBackoff);
}
public static ConcreteBackOffer newRawKVBackOff() {
return new ConcreteBackOffer(rawkvMaxBackoff);
}
public static ConcreteBackOffer newTsoBackOff() {
return new ConcreteBackOffer(tsoMaxBackoff);
}
public static ConcreteBackOffer create(BackOffer source) {
return new ConcreteBackOffer(((ConcreteBackOffer) source));
}
private int totalSleep;
private ConcreteBackOffer(int maxSleep) {
Preconditions.checkArgument(maxSleep >= 0, "Max sleep time cannot be less than 0.");
@ -78,6 +47,38 @@ public class ConcreteBackOffer implements BackOffer {
this.backOffFunctionMap = source.backOffFunctionMap;
}
public static ConcreteBackOffer newCustomBackOff(int maxSleep) {
return new ConcreteBackOffer(maxSleep);
}
public static ConcreteBackOffer newScannerNextMaxBackOff() {
return new ConcreteBackOffer(SCANNER_NEXT_MAX_BACKOFF);
}
public static ConcreteBackOffer newBatchGetMaxBackOff() {
return new ConcreteBackOffer(BATCH_GET_MAX_BACKOFF);
}
public static ConcreteBackOffer newCopNextMaxBackOff() {
return new ConcreteBackOffer(COP_NEXT_MAX_BACKOFF);
}
public static ConcreteBackOffer newGetBackOff() {
return new ConcreteBackOffer(GET_MAX_BACKOFF);
}
public static ConcreteBackOffer newRawKVBackOff() {
return new ConcreteBackOffer(RAWKV_MAX_BACKOFF);
}
public static ConcreteBackOffer newTsoBackOff() {
return new ConcreteBackOffer(TSO_MAX_BACKOFF);
}
public static ConcreteBackOffer create(BackOffer source) {
return new ConcreteBackOffer(((ConcreteBackOffer) source));
}
/**
* Creates a back off func which implements exponential back off with optional jitters according
* to different back off strategies. See http://www.awsarchitectureblog.com/2015/03/backoff.html
@ -86,7 +87,6 @@ public class ConcreteBackOffer implements BackOffer {
BackOffFunction backOffFunction = null;
switch (funcType) {
case BoUpdateLeader:
// fix: reference from go client
backOffFunction = BackOffFunction.create(1, 10, BackOffStrategy.NoJitter);
break;
case BoTxnLockFast:
@ -96,23 +96,19 @@ public class ConcreteBackOffer implements BackOffer {
backOffFunction = BackOffFunction.create(2000, 10000, BackOffStrategy.EqualJitter);
break;
case BoRegionMiss:
// fix: reference from go client
// change base time to 2ms, because it may recover soon.
backOffFunction = BackOffFunction.create(2, 500, BackOffStrategy.NoJitter);
backOffFunction = BackOffFunction.create(100, 500, BackOffStrategy.NoJitter);
break;
case BoTxnLock:
backOffFunction = BackOffFunction.create(200, 3000, BackOffStrategy.EqualJitter);
break;
case BoPDRPC:
//
backOffFunction = BackOffFunction.create(500, 3000, BackOffStrategy.EqualJitter);
break;
case BoTiKVRPC:
// fix: reference from go client
backOffFunction = BackOffFunction.create(100, 2000, BackOffStrategy.EqualJitter);
break;
case BoStoreNotMatch:
backOffFunction = BackOffFunction.create(100, 3000, BackOffStrategy.EqualJitter);
case BoTxnNotFound:
backOffFunction = BackOffFunction.create(2, 500, BackOffStrategy.NoJitter);
break;
}
return backOffFunction;
@ -120,11 +116,17 @@ public class ConcreteBackOffer implements BackOffer {
@Override
public void doBackOff(BackOffFunction.BackOffFuncType funcType, Exception err) {
doBackOffWithMaxSleep(funcType, -1, err);
}
@Override
public void doBackOffWithMaxSleep(
BackOffFunction.BackOffFuncType funcType, long maxSleepMs, Exception err) {
BackOffFunction backOffFunction =
backOffFunctionMap.computeIfAbsent(funcType, this::createBackOffFunc);
// Back off will be done here
totalSleep += backOffFunction.doBackOff();
totalSleep += backOffFunction.doBackOff(maxSleepMs);
logger.debug(
String.format(
"%s, retry later(totalSleep %dms, maxSleep %dms)",
@ -133,7 +135,7 @@ public class ConcreteBackOffer implements BackOffer {
if (maxSleep > 0 && totalSleep >= maxSleep) {
StringBuilder errMsg =
new StringBuilder(
String.format("backoffer.maxSleep %dms is exceeded, errors:", maxSleep));
String.format("BackOffer.maxSleep %dms is exceeded, errors:", maxSleep));
for (int i = 0; i < errors.size(); i++) {
Exception curErr = errors.get(i);
// Print only last 3 errors for non-DEBUG log levels.

View File

@ -1,15 +1,30 @@
/*
* Copyright 2020 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.tikv.common.util;
public final class TsoUtils {
private static final long physicalShiftBits = 18;
import org.tikv.common.meta.TiTimestamp;
public final class TsoUtils {
public static boolean isExpired(long lockTS, long ttl) {
// Because the UNIX time in milliseconds is in long style and will
// not exceed to become the negative number, so the comparison is correct
return System.currentTimeMillis() >= extractPhysical(lockTS) + ttl;
return untilExpired(lockTS, ttl) <= 0;
}
private static long extractPhysical(long ts) {
return ts >> physicalShiftBits;
public static long untilExpired(long lockTS, long ttl) {
return TiTimestamp.extractPhysical(lockTS) + ttl - System.currentTimeMillis();
}
}

View File

@ -30,7 +30,8 @@ import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import org.apache.log4j.Logger;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.tikv.common.TiConfiguration;
import org.tikv.common.exception.GrpcException;
import org.tikv.common.exception.TiKVException;
@ -47,7 +48,7 @@ public class RawKVClient implements AutoCloseable {
private final RegionStoreClientBuilder clientBuilder;
private final TiConfiguration conf;
private final ExecutorCompletionService<Object> completionService;
private static final Logger logger = Logger.getLogger(RawKVClient.class);
private static final Logger logger = LoggerFactory.getLogger(RawKVClient.class);
private static final int MAX_RETRY_LIMIT = 3;
// https://www.github.com/pingcap/tidb/blob/master/store/tikv/rawkv.go
@ -55,7 +56,8 @@ public class RawKVClient implements AutoCloseable {
private static final int RAW_BATCH_PUT_SIZE = 16 * 1024;
private static final int RAW_BATCH_PAIR_COUNT = 512;
private static final TiKVException ERR_RETRY_LIMIT_EXCEEDED = new GrpcException("retry is exhausted. retry exceeds " + MAX_RETRY_LIMIT + "attempts");
private static final TiKVException ERR_RETRY_LIMIT_EXCEEDED =
new GrpcException("retry is exhausted. retry exceeds " + MAX_RETRY_LIMIT + "attempts");
private static final TiKVException ERR_MAX_SCAN_LIMIT_EXCEEDED =
new TiKVException("limit should be less than MAX_RAW_SCAN_LIMIT");
@ -291,7 +293,7 @@ public class RawKVClient implements AutoCloseable {
}
try {
for (int i = 0; i < batches.size(); i++) {
completionService.take().get(BackOffer.rawkvMaxBackoff, TimeUnit.SECONDS);
completionService.take().get(BackOffer.RAWKV_MAX_BACKOFF, TimeUnit.SECONDS);
}
} catch (InterruptedException e) {
Thread.currentThread().interrupt();

View File

@ -0,0 +1,123 @@
/*
*
* Copyright 2020 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*
*/
package org.tikv.txn;
import java.util.List;
import org.tikv.common.PDClient;
import org.tikv.common.StoreVersion;
import org.tikv.common.TiConfiguration;
import org.tikv.common.Version;
import org.tikv.common.exception.KeyException;
import org.tikv.common.region.RegionManager;
import org.tikv.common.region.RegionStoreClient;
import org.tikv.common.region.TiRegion;
import org.tikv.common.util.BackOffer;
import org.tikv.common.util.ChannelFactory;
import org.tikv.kvproto.Kvrpcpb;
import org.tikv.kvproto.Metapb;
import org.tikv.kvproto.TikvGrpc;
public interface AbstractLockResolverClient {
/** ResolvedCacheSize is max number of cached txn status. */
long RESOLVED_TXN_CACHE_SIZE = 2048;
/** transaction involves keys exceed this threshold can be treated as `big transaction`. */
long BIG_TXN_THRESHOLD = 16;
static Lock extractLockFromKeyErr(Kvrpcpb.KeyError keyError) {
if (keyError.hasLocked()) {
return new Lock(keyError.getLocked());
}
if (keyError.hasConflict()) {
Kvrpcpb.WriteConflict conflict = keyError.getConflict();
throw new KeyException(
String.format(
"scan meet key conflict on primary key %s at commit ts %s",
conflict.getPrimary(), conflict.getConflictTs()));
}
if (!keyError.getRetryable().isEmpty()) {
throw new KeyException(
String.format("tikv restart txn %s", keyError.getRetryableBytes().toStringUtf8()));
}
if (!keyError.getAbort().isEmpty()) {
throw new KeyException(
String.format("tikv abort txn %s", keyError.getAbortBytes().toStringUtf8()));
}
throw new KeyException(
String.format("unexpected key error meets and it is %s", keyError.toString()));
}
static AbstractLockResolverClient getInstance(
Metapb.Store store,
TiConfiguration conf,
TiRegion region,
TikvGrpc.TikvBlockingStub blockingStub,
TikvGrpc.TikvStub asyncStub,
ChannelFactory channelFactory,
RegionManager regionManager,
PDClient pdClient,
RegionStoreClient.RegionStoreClientBuilder clientBuilder) {
if (StoreVersion.compareTo(store.getVersion(), Version.RESOLVE_LOCK_V3) < 0) {
return new LockResolverClientV2(
conf, region, blockingStub, asyncStub, channelFactory, regionManager);
} else if (StoreVersion.compareTo(store.getVersion(), Version.RESOLVE_LOCK_V4) < 0) {
return new LockResolverClientV3(
conf,
region,
blockingStub,
asyncStub,
channelFactory,
regionManager,
pdClient,
clientBuilder);
} else {
return new LockResolverClientV4(
conf,
region,
blockingStub,
asyncStub,
channelFactory,
regionManager,
pdClient,
clientBuilder);
}
}
String getVersion();
/**
* ResolveLocks tries to resolve Locks. The resolving process is in 3 steps: 1) Use the `lockTTL`
* to pick up all expired locks. Only locks that are old enough are considered orphan locks and
* will be handled later. If all locks are expired then all locks will be resolved so true will be
* returned, otherwise caller should sleep a while before retry. 2) For each lock, query the
* primary key to get txn(which left the lock)'s commit status. 3) Send `ResolveLock` cmd to the
* lock's region to resolve all locks belong to the same transaction.
*
* @param bo
* @param callerStartTS
* @param locks
* @param forWrite
* @return msBeforeTxnExpired: 0 means all locks are resolved
*/
ResolveLockResult resolveLocks(
BackOffer bo, long callerStartTS, List<Lock> locks, boolean forWrite);
}

View File

@ -0,0 +1,62 @@
/*
* Copyright 2017 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.tikv.txn;
import com.google.protobuf.ByteString;
import java.util.ArrayList;
import java.util.List;
import org.tikv.common.region.TiRegion;
import org.tikv.kvproto.Metapb;
public class BatchKeys {
private final TiRegion region;
private final Metapb.Store store;
private List<ByteString> keys;
private final int sizeInBytes;
public BatchKeys(
TiRegion region, Metapb.Store store, List<ByteString> keysInput, int sizeInBytes) {
this.region = region;
this.store = store;
this.keys = new ArrayList<>();
this.keys.addAll(keysInput);
this.sizeInBytes = sizeInBytes;
}
public List<ByteString> getKeys() {
return keys;
}
public void setKeys(List<ByteString> keys) {
this.keys = keys;
}
public TiRegion getRegion() {
return region;
}
public Metapb.Store getStore() {
return store;
}
public int getSizeInBytes() {
return sizeInBytes;
}
public float getSizeInKB() {
return ((float) sizeInBytes) / 1024;
}
}

View File

@ -0,0 +1,52 @@
/*
* Copyright 2019 The TiKV Project Authors
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.tikv.txn;
public class ClientRPCResult {
private boolean success;
private boolean retry;
private Exception exception;
public ClientRPCResult(boolean success, boolean retry, Exception exception) {
this.success = success;
this.retry = retry;
this.exception = exception;
}
public boolean isSuccess() {
return success;
}
public void setSuccess(boolean success) {
this.success = success;
}
public boolean isRetry() {
return retry;
}
public void setRetry(boolean retry) {
this.retry = retry;
}
public Exception getException() {
return exception;
}
public void setException(Exception exception) {
this.exception = exception;
}
}

View File

@ -0,0 +1,41 @@
/*
* Copyright 2017 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.tikv.txn;
import com.google.protobuf.ByteString;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.tikv.common.region.TiRegion;
import org.tikv.common.util.Pair;
import org.tikv.kvproto.Metapb;
public class GroupKeyResult {
private Map<Pair<TiRegion, Metapb.Store>, List<ByteString>> groupsResult;
public GroupKeyResult() {
this.groupsResult = new HashMap<>();
}
public Map<Pair<TiRegion, Metapb.Store>, List<ByteString>> getGroupsResult() {
return groupsResult;
}
public void setGroupsResult(Map<Pair<TiRegion, Metapb.Store>, List<ByteString>> groupsResult) {
this.groupsResult = groupsResult;
}
}

View File

@ -0,0 +1,271 @@
/*
* Copyright 2020 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.tikv.txn;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.google.protobuf.ByteString;
import java.util.*;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.stream.Collectors;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.tikv.common.TiConfiguration;
import org.tikv.common.exception.GrpcException;
import org.tikv.common.exception.TiKVException;
import org.tikv.common.operation.iterator.ConcreteScanIterator;
import org.tikv.common.region.RegionStoreClient;
import org.tikv.common.region.RegionStoreClient.RegionStoreClientBuilder;
import org.tikv.common.region.TiRegion;
import org.tikv.common.util.BackOffFunction;
import org.tikv.common.util.BackOffer;
import org.tikv.common.util.ConcreteBackOffer;
import org.tikv.kvproto.Kvrpcpb;
public class KVClient implements AutoCloseable {
private static final Logger logger = LoggerFactory.getLogger(KVClient.class);
private static final int BATCH_GET_SIZE = 16 * 1024;
private final RegionStoreClientBuilder clientBuilder;
private final TiConfiguration conf;
private final ExecutorService executorService;
public KVClient(TiConfiguration conf, RegionStoreClientBuilder clientBuilder) {
Objects.requireNonNull(conf, "conf is null");
Objects.requireNonNull(clientBuilder, "clientBuilder is null");
this.conf = conf;
this.clientBuilder = clientBuilder;
executorService =
Executors.newFixedThreadPool(
conf.getKvClientConcurrency(),
new ThreadFactoryBuilder().setNameFormat("kvclient-pool-%d").setDaemon(true).build());
}
@Override
public void close() {
if (executorService != null) {
executorService.shutdownNow();
}
}
/**
* Get a key-value pair from TiKV if key exists
*
* @param key key
* @return a ByteString value if key exists, ByteString.EMPTY if key does not exist
*/
public ByteString get(ByteString key, long version) throws GrpcException {
BackOffer backOffer = ConcreteBackOffer.newGetBackOff();
while (true) {
RegionStoreClient client = clientBuilder.build(key);
try {
return client.get(backOffer, key, version);
} catch (final TiKVException e) {
backOffer.doBackOff(BackOffFunction.BackOffFuncType.BoRegionMiss, e);
}
}
}
/**
* Get a set of key-value pair by keys from TiKV
*
* @param backOffer
* @param keys
* @param version
* @return
* @throws GrpcException
*/
public List<Kvrpcpb.KvPair> batchGet(BackOffer backOffer, List<ByteString> keys, long version)
throws GrpcException {
return doSendBatchGet(backOffer, keys, version);
}
/**
* Scan key-value pairs from TiKV in range [startKey, endKey)
*
* @param startKey start key, inclusive
* @param endKey end key, exclusive
* @return list of key-value pairs in range
*/
public List<Kvrpcpb.KvPair> scan(ByteString startKey, ByteString endKey, long version)
throws GrpcException {
Iterator<Kvrpcpb.KvPair> iterator =
scanIterator(conf, clientBuilder, startKey, endKey, version);
List<Kvrpcpb.KvPair> result = new ArrayList<>();
iterator.forEachRemaining(result::add);
return result;
}
/**
* Scan key-value pairs from TiKV in range [startKey, ), maximum to `limit` pairs
*
* @param startKey start key, inclusive
* @param limit limit of kv pairs
* @return list of key-value pairs in range
*/
public List<Kvrpcpb.KvPair> scan(ByteString startKey, long version, int limit)
throws GrpcException {
Iterator<Kvrpcpb.KvPair> iterator = scanIterator(conf, clientBuilder, startKey, version, limit);
List<Kvrpcpb.KvPair> result = new ArrayList<>();
iterator.forEachRemaining(result::add);
return result;
}
public List<Kvrpcpb.KvPair> scan(ByteString startKey, long version) throws GrpcException {
return scan(startKey, version, Integer.MAX_VALUE);
}
private List<Kvrpcpb.KvPair> doSendBatchGet(
BackOffer backOffer, List<ByteString> keys, long version) {
ExecutorCompletionService<List<Kvrpcpb.KvPair>> completionService =
new ExecutorCompletionService<>(executorService);
Map<TiRegion, List<ByteString>> groupKeys = groupKeysByRegion(keys);
List<Batch> batches = new ArrayList<>();
for (Map.Entry<TiRegion, List<ByteString>> entry : groupKeys.entrySet()) {
appendBatches(batches, entry.getKey(), entry.getValue(), BATCH_GET_SIZE);
}
for (Batch batch : batches) {
BackOffer singleBatchBackOffer = ConcreteBackOffer.create(backOffer);
completionService.submit(
() -> doSendBatchGetInBatchesWithRetry(singleBatchBackOffer, batch, version));
}
try {
List<Kvrpcpb.KvPair> result = new ArrayList<>();
for (int i = 0; i < batches.size(); i++) {
result.addAll(completionService.take().get());
}
return result;
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new TiKVException("Current thread interrupted.", e);
} catch (ExecutionException e) {
throw new TiKVException("Execution exception met.", e);
}
}
private List<Kvrpcpb.KvPair> doSendBatchGetInBatchesWithRetry(
BackOffer backOffer, Batch batch, long version) {
TiRegion oldRegion = batch.region;
TiRegion currentRegion =
clientBuilder.getRegionManager().getRegionByKey(oldRegion.getStartKey());
if (oldRegion.equals(currentRegion)) {
RegionStoreClient client = clientBuilder.build(batch.region);
try {
return client.batchGet(backOffer, batch.keys, version);
} catch (final TiKVException e) {
backOffer.doBackOff(BackOffFunction.BackOffFuncType.BoRegionMiss, e);
clientBuilder.getRegionManager().invalidateRegion(batch.region.getId());
logger.warn("ReSplitting ranges for BatchGetRequest", e);
// retry
return doSendBatchGetWithRefetchRegion(backOffer, batch, version);
}
} else {
return doSendBatchGetWithRefetchRegion(backOffer, batch, version);
}
}
private List<Kvrpcpb.KvPair> doSendBatchGetWithRefetchRegion(
BackOffer backOffer, Batch batch, long version) {
Map<TiRegion, List<ByteString>> groupKeys = groupKeysByRegion(batch.keys);
List<Batch> retryBatches = new ArrayList<>();
for (Map.Entry<TiRegion, List<ByteString>> entry : groupKeys.entrySet()) {
appendBatches(retryBatches, entry.getKey(), entry.getValue(), BATCH_GET_SIZE);
}
ArrayList<Kvrpcpb.KvPair> results = new ArrayList<>();
for (Batch retryBatch : retryBatches) {
// recursive calls
List<Kvrpcpb.KvPair> batchResult =
doSendBatchGetInBatchesWithRetry(backOffer, retryBatch, version);
results.addAll(batchResult);
}
return results;
}
/**
* Append batch to list and split them according to batch limit
*
* @param batches a grouped batch
* @param region region
* @param keys keys
* @param batchGetMaxSizeInByte batch max limit
*/
private void appendBatches(
List<Batch> batches, TiRegion region, List<ByteString> keys, int batchGetMaxSizeInByte) {
int start;
int end;
if (keys == null) {
return;
}
int len = keys.size();
for (start = 0; start < len; start = end) {
int size = 0;
for (end = start; end < len && size < batchGetMaxSizeInByte; end++) {
size += keys.get(end).size();
}
Batch batch = new Batch(region, keys.subList(start, end));
batches.add(batch);
}
}
/**
* Group by list of keys according to its region
*
* @param keys keys
* @return a mapping of keys and their region
*/
private Map<TiRegion, List<ByteString>> groupKeysByRegion(List<ByteString> keys) {
return keys.stream()
.collect(Collectors.groupingBy(clientBuilder.getRegionManager()::getRegionByKey));
}
private Iterator<Kvrpcpb.KvPair> scanIterator(
TiConfiguration conf,
RegionStoreClientBuilder builder,
ByteString startKey,
ByteString endKey,
long version) {
return new ConcreteScanIterator(conf, builder, startKey, endKey, version);
}
private Iterator<Kvrpcpb.KvPair> scanIterator(
TiConfiguration conf,
RegionStoreClientBuilder builder,
ByteString startKey,
long version,
int limit) {
return new ConcreteScanIterator(conf, builder, startKey, version, limit);
}
/** A Batch containing the region and a list of keys to send */
private static final class Batch {
private final TiRegion region;
private final List<ByteString> keys;
Batch(TiRegion region, List<ByteString> keys) {
this.region = region;
this.keys = keys;
}
}
}

View File

@ -21,17 +21,23 @@ import com.google.protobuf.ByteString;
import org.tikv.kvproto.Kvrpcpb;
public class Lock {
private static final long DEFAULT_LOCK_TTL = 3000;
private final long txnID;
private final long ttl;
private final ByteString key;
private final ByteString primary;
private static final long defaultLockTTL = 3000;
private final long txnSize;
private final Kvrpcpb.Op lockType;
private final long lockForUpdateTs;
public Lock(Kvrpcpb.LockInfo l) {
txnID = l.getLockVersion();
key = l.getKey();
primary = l.getPrimaryLock();
ttl = l.getLockTtl() == 0 ? defaultLockTTL : l.getLockTtl();
ttl = l.getLockTtl() == 0 ? DEFAULT_LOCK_TTL : l.getLockTtl();
txnSize = l.getTxnSize();
lockType = l.getLockType();
lockForUpdateTs = l.getLockForUpdateTs();
}
public long getTxnID() {
@ -49,4 +55,16 @@ public class Lock {
public ByteString getPrimary() {
return primary;
}
public long getTxnSize() {
return txnSize;
}
public Kvrpcpb.Op getLockType() {
return lockType;
}
public long getLockForUpdateTs() {
return lockForUpdateTs;
}
}

View File

@ -20,7 +20,6 @@ package org.tikv.txn;
import static org.tikv.common.util.BackOffFunction.BackOffFuncType.BoRegionMiss;
import com.google.protobuf.ByteString;
import io.grpc.ManagedChannel;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
@ -32,13 +31,14 @@ import java.util.Set;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Supplier;
import org.apache.log4j.Logger;
import org.tikv.common.AbstractGRPCClient;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.tikv.common.TiConfiguration;
import org.tikv.common.exception.KeyException;
import org.tikv.common.exception.RegionException;
import org.tikv.common.exception.TiClientInternalException;
import org.tikv.common.operation.KVErrorHandler;
import org.tikv.common.region.RegionErrorReceiver;
import org.tikv.common.region.AbstractRegionStoreClient;
import org.tikv.common.region.RegionManager;
import org.tikv.common.region.TiRegion;
import org.tikv.common.region.TiRegion.RegionVerID;
@ -49,52 +49,39 @@ import org.tikv.kvproto.Kvrpcpb.CleanupRequest;
import org.tikv.kvproto.Kvrpcpb.CleanupResponse;
import org.tikv.kvproto.Kvrpcpb.ResolveLockRequest;
import org.tikv.kvproto.Kvrpcpb.ResolveLockResponse;
import org.tikv.kvproto.Metapb.Store;
import org.tikv.kvproto.TikvGrpc;
import org.tikv.kvproto.TikvGrpc.TikvBlockingStub;
import org.tikv.kvproto.TikvGrpc.TikvStub;
// LockResolver resolves locks and also caches resolved txn status.
public class LockResolverClient extends AbstractGRPCClient<TikvBlockingStub, TikvStub>
implements RegionErrorReceiver {
// ResolvedCacheSize is max number of cached txn status.
private static final long RESOLVED_TXN_CACHE_SIZE = 2048;
// By default, locks after 3000ms is considered unusual (the client created the
// lock might be dead). Other client may cleanup this kind of lock.
// For locks created recently, we will do backoff and retry.
private static final long DEFAULT_LOCK_TTL = 3000;
private static final long MAX_LOCK_TTL = 120000;
// ttl = ttlFactor * sqrt(writeSizeInMiB)
private static final long TTL_FACTOR = 6000;
private static final Logger logger = Logger.getLogger(LockResolverClient.class);
/** Before v3.0.5 TiDB uses the ttl on secondary lock. */
public class LockResolverClientV2 extends AbstractRegionStoreClient
implements AbstractLockResolverClient {
private static final Logger logger = LoggerFactory.getLogger(LockResolverClientV2.class);
private final ReadWriteLock readWriteLock;
// Note: Because the internal of long is same as unsigned_long
// and Txn id are never changed. Be careful to compare between two tso
// the `resolved` mapping is as {@code Map<TxnId, TxnStatus>}
// TxnStatus represents a txn's final status. It should be Commit or Rollback.
// if TxnStatus > 0, means the commit ts, otherwise abort
private final Map<Long, Long> resolved;
// the list is chain of txn for O(1) lru cache
private final Queue<Long> recentResolved;
private TikvBlockingStub blockingStub;
private TikvStub asyncStub;
private TiRegion region;
private final RegionManager regionManager;
public LockResolverClient(
/**
* Note: Because the internal of long is same as unsigned_long and Txn id are never changed. Be
* careful to compare between two tso the `resolved` mapping is as {@code Map<TxnId, TxnStatus>}
* TxnStatus represents a txn's final status. It should be Commit or Rollback. if TxnStatus > 0,
* means the commit ts, otherwise abort
*/
private final Map<Long, Long> resolved;
/** the list is chain of txn for O(1) lru cache */
private final Queue<Long> recentResolved;
public LockResolverClientV2(
TiConfiguration conf,
TiRegion region,
TikvBlockingStub blockingStub,
TikvStub asyncStub,
ChannelFactory channelFactory,
RegionManager regionManager) {
super(conf, channelFactory);
super(conf, region, channelFactory, blockingStub, asyncStub, regionManager);
resolved = new HashMap<>();
recentResolved = new LinkedList<>();
readWriteLock = new ReentrantReadWriteLock();
this.blockingStub = blockingStub;
this.regionManager = regionManager;
this.asyncStub = asyncStub;
}
private void saveResolved(long txnID, long status) {
@ -124,7 +111,7 @@ public class LockResolverClient extends AbstractGRPCClient<TikvBlockingStub, Tik
}
}
public Long getTxnStatus(BackOffer bo, Long txnID, ByteString primary) {
private Long getTxnStatus(BackOffer bo, Long txnID, ByteString primary) {
Long status = getResolved(txnID);
if (status != null) {
@ -146,12 +133,26 @@ public class LockResolverClient extends AbstractGRPCClient<TikvBlockingStub, Tik
new KVErrorHandler<>(
regionManager,
this,
this,
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null);
CleanupResponse resp = callWithRetry(bo, TikvGrpc.METHOD_KV_CLEANUP, factory, handler);
resp -> resp.hasRegionError() ? resp.getRegionError() : null,
resp -> resp.hasError() ? resp.getError() : null,
resolveLockResult -> null,
0L,
false);
CleanupResponse resp = callWithRetry(bo, TikvGrpc.getKvCleanupMethod(), factory, handler);
status = 0L;
if (resp == null) {
logger.error("getKvCleanupMethod failed without a cause");
regionManager.onRequestFail(region);
bo.doBackOff(
BoRegionMiss,
new TiClientInternalException("getKvCleanupMethod failed without a cause"));
continue;
}
if (resp.hasRegionError()) {
bo.doBackOff(BoRegionMiss, new RegionException(resp.getRegionError()));
continue;
@ -171,16 +172,22 @@ public class LockResolverClient extends AbstractGRPCClient<TikvBlockingStub, Tik
}
}
// ResolveLocks tries to resolve Locks. The resolving process is in 3 steps:
// 1) Use the `lockTTL` to pick up all expired locks. Only locks that are old
// enough are considered orphan locks and will be handled later. If all locks
// are expired then all locks will be resolved so true will be returned, otherwise
// caller should sleep a while before retry.
// 2) For each lock, query the primary key to get txn(which left the lock)'s
// commit status.
// 3) Send `ResolveLock` cmd to the lock's region to resolve all locks belong to
// the same transaction.
public boolean resolveLocks(BackOffer bo, List<Lock> locks) {
@Override
public String getVersion() {
return "V2";
}
@Override
public ResolveLockResult resolveLocks(
BackOffer bo, long callerStartTS, List<Lock> locks, boolean forWrite) {
if (doResolveLocks(bo, locks)) {
return new ResolveLockResult(0L);
} else {
return new ResolveLockResult(10000L);
}
}
private boolean doResolveLocks(BackOffer bo, List<Lock> locks) {
if (locks.isEmpty()) {
return true;
}
@ -197,7 +204,7 @@ public class LockResolverClient extends AbstractGRPCClient<TikvBlockingStub, Tik
}
// TxnID -> []Region, record resolved Regions.
// TODO: Maybe put it in all LockResolverClient and share by all txns.
// TODO: Maybe put it in all LockResolverClientV2 and share by all txns.
Map<Long, Set<RegionVerID>> cleanTxns = new HashMap<>();
for (Lock l : expiredLocks) {
Long status = getTxnStatus(bo, l.getTxnID(), l.getPrimary());
@ -243,16 +250,23 @@ public class LockResolverClient extends AbstractGRPCClient<TikvBlockingStub, Tik
new KVErrorHandler<>(
regionManager,
this,
this,
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null);
resp -> resp.hasRegionError() ? resp.getRegionError() : null,
resp -> resp.hasError() ? resp.getError() : null,
resolveLockResult -> null,
0L,
false);
ResolveLockResponse resp =
callWithRetry(bo, TikvGrpc.METHOD_KV_RESOLVE_LOCK, factory, handler);
callWithRetry(bo, TikvGrpc.getKvResolveLockMethod(), factory, handler);
if (resp.hasError()) {
logger.error(
String.format("unexpected resolveLock err: %s, lock: %s", resp.getError(), lock));
throw new KeyException(resp.getError());
if (resp == null) {
logger.error("getKvResolveLockMethod failed without a cause");
regionManager.onRequestFail(region);
bo.doBackOff(
BoRegionMiss,
new TiClientInternalException("getKvResolveLockMethod failed without a cause"));
continue;
}
if (resp.hasRegionError()) {
@ -260,55 +274,14 @@ public class LockResolverClient extends AbstractGRPCClient<TikvBlockingStub, Tik
continue;
}
if (resp.hasError()) {
logger.error(
String.format("unexpected resolveLock err: %s, lock: %s", resp.getError(), lock));
throw new KeyException(resp.getError());
}
cleanRegion.add(region.getVerID());
return;
}
}
@Override
protected TikvBlockingStub getBlockingStub() {
return blockingStub.withDeadlineAfter(getConf().getTimeout(), getConf().getTimeoutUnit());
}
@Override
protected TikvStub getAsyncStub() {
return asyncStub.withDeadlineAfter(getConf().getTimeout(), getConf().getTimeoutUnit());
}
@Override
public void close() throws Exception {}
/**
* onNotLeader deals with NotLeaderError and returns whether re-splitting key range is needed
*
* @param newStore the new store presented by NotLeader Error
* @return false when re-split is needed.
*/
@Override
public boolean onNotLeader(Store newStore) {
if (logger.isDebugEnabled()) {
logger.debug(region + ", new leader = " + newStore.getId());
}
TiRegion cachedRegion = regionManager.getRegionById(region.getId());
// When switch leader fails or the region changed its key range,
// it would be necessary to re-split task's key range for new region.
if (!region.getStartKey().equals(cachedRegion.getStartKey())
|| !region.getEndKey().equals(cachedRegion.getEndKey())) {
return false;
}
region = cachedRegion;
String addressStr = newStore.getAddress();
ManagedChannel channel = channelFactory.getChannel(addressStr);
blockingStub = TikvGrpc.newBlockingStub(channel);
asyncStub = TikvGrpc.newStub(channel);
return true;
}
@Override
public void onStoreNotMatch(Store store) {
String addressStr = store.getAddress();
ManagedChannel channel = channelFactory.getChannel(addressStr);
blockingStub = TikvGrpc.newBlockingStub(channel);
asyncStub = TikvGrpc.newStub(channel);
}
}

View File

@ -0,0 +1,325 @@
/*
*
* Copyright 2020 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*
*/
package org.tikv.txn;
import static org.tikv.common.util.BackOffFunction.BackOffFuncType.BoRegionMiss;
import com.google.protobuf.ByteString;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Supplier;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.tikv.common.PDClient;
import org.tikv.common.TiConfiguration;
import org.tikv.common.exception.KeyException;
import org.tikv.common.exception.RegionException;
import org.tikv.common.exception.TiClientInternalException;
import org.tikv.common.operation.KVErrorHandler;
import org.tikv.common.region.AbstractRegionStoreClient;
import org.tikv.common.region.RegionManager;
import org.tikv.common.region.RegionStoreClient;
import org.tikv.common.region.TiRegion;
import org.tikv.common.region.TiRegion.RegionVerID;
import org.tikv.common.util.BackOffer;
import org.tikv.common.util.ChannelFactory;
import org.tikv.common.util.TsoUtils;
import org.tikv.kvproto.Kvrpcpb;
import org.tikv.kvproto.Kvrpcpb.CleanupRequest;
import org.tikv.kvproto.Kvrpcpb.CleanupResponse;
import org.tikv.kvproto.TikvGrpc;
import org.tikv.kvproto.TikvGrpc.TikvBlockingStub;
import org.tikv.kvproto.TikvGrpc.TikvStub;
/** Since v3.0.5 TiDB ignores the ttl on secondary lock and will use the ttl on primary key. */
public class LockResolverClientV3 extends AbstractRegionStoreClient
implements AbstractLockResolverClient {
private static final Logger logger = LoggerFactory.getLogger(LockResolverClientV3.class);
private final ReadWriteLock readWriteLock;
/**
* Note: Because the internal of long is same as unsigned_long and Txn id are never changed. Be
* careful to compare between two tso the `resolved` mapping is as {@code Map<TxnId, TxnStatus>}
* TxnStatus represents a txn's final status. It should be Commit or Rollback. if TxnStatus > 0,
* means the commit ts, otherwise abort
*/
private final Map<Long, TxnStatus> resolved;
/** the list is chain of txn for O(1) lru cache */
private final Queue<Long> recentResolved;
private final PDClient pdClient;
private final RegionStoreClient.RegionStoreClientBuilder clientBuilder;
public LockResolverClientV3(
TiConfiguration conf,
TiRegion region,
TikvBlockingStub blockingStub,
TikvStub asyncStub,
ChannelFactory channelFactory,
RegionManager regionManager,
PDClient pdClient,
RegionStoreClient.RegionStoreClientBuilder clientBuilder) {
super(conf, region, channelFactory, blockingStub, asyncStub, regionManager);
resolved = new HashMap<>();
recentResolved = new LinkedList<>();
readWriteLock = new ReentrantReadWriteLock();
this.pdClient = pdClient;
this.clientBuilder = clientBuilder;
}
@Override
public String getVersion() {
return "V3";
}
@Override
public ResolveLockResult resolveLocks(
BackOffer bo, long callerStartTS, List<Lock> locks, boolean forWrite) {
TxnExpireTime msBeforeTxnExpired = new TxnExpireTime();
if (locks.isEmpty()) {
return new ResolveLockResult(msBeforeTxnExpired.value());
}
List<Lock> expiredLocks = new ArrayList<>();
for (Lock lock : locks) {
if (TsoUtils.isExpired(lock.getTxnID(), lock.getTtl())) {
expiredLocks.add(lock);
} else {
msBeforeTxnExpired.update(lock.getTtl());
}
}
if (expiredLocks.isEmpty()) {
return new ResolveLockResult(msBeforeTxnExpired.value());
}
Map<Long, Set<RegionVerID>> cleanTxns = new HashMap<>();
for (Lock l : expiredLocks) {
TxnStatus status = getTxnStatusFromLock(bo, l);
if (status.getTtl() == 0) {
Set<RegionVerID> cleanRegion =
cleanTxns.computeIfAbsent(l.getTxnID(), k -> new HashSet<>());
resolveLock(bo, l, status, cleanRegion);
} else {
long msBeforeLockExpired = TsoUtils.untilExpired(l.getTxnID(), status.getTtl());
msBeforeTxnExpired.update(msBeforeLockExpired);
}
}
return new ResolveLockResult(msBeforeTxnExpired.value());
}
private void resolveLock(
BackOffer bo, Lock lock, TxnStatus txnStatus, Set<RegionVerID> cleanRegion) {
boolean cleanWholeRegion = lock.getTxnSize() >= BIG_TXN_THRESHOLD;
while (true) {
region = regionManager.getRegionByKey(lock.getKey());
if (cleanRegion.contains(region.getVerID())) {
return;
}
Kvrpcpb.ResolveLockRequest.Builder builder =
Kvrpcpb.ResolveLockRequest.newBuilder()
.setContext(region.getContext())
.setStartVersion(lock.getTxnID());
if (txnStatus.isCommitted()) {
// txn is committed with commitTS txnStatus
builder.setCommitVersion(txnStatus.getCommitTS());
}
if (lock.getTxnSize() < BIG_TXN_THRESHOLD) {
// Only resolve specified keys when it is a small transaction,
// prevent from scanning the whole region in this case.
builder.addKeys(lock.getKey());
}
Supplier<Kvrpcpb.ResolveLockRequest> factory = builder::build;
KVErrorHandler<Kvrpcpb.ResolveLockResponse> handler =
new KVErrorHandler<>(
regionManager,
this,
this,
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null,
resp -> resp.hasError() ? resp.getError() : null,
resolveLockResult -> null,
0L,
false);
Kvrpcpb.ResolveLockResponse resp =
callWithRetry(bo, TikvGrpc.getKvResolveLockMethod(), factory, handler);
if (resp == null) {
logger.error("getKvResolveLockMethod failed without a cause");
regionManager.onRequestFail(region);
bo.doBackOff(
BoRegionMiss,
new TiClientInternalException("getKvResolveLockMethod failed without a cause"));
continue;
}
if (resp.hasRegionError()) {
bo.doBackOff(BoRegionMiss, new RegionException(resp.getRegionError()));
continue;
}
if (resp.hasError()) {
logger.error(
String.format("unexpected resolveLock err: %s, lock: %s", resp.getError(), lock));
throw new KeyException(resp.getError());
}
if (cleanWholeRegion) {
cleanRegion.add(region.getVerID());
}
return;
}
}
private TxnStatus getTxnStatusFromLock(BackOffer bo, Lock lock) {
// NOTE: l.TTL = 0 is a special protocol!!!
// When the pessimistic txn prewrite meets locks of a txn, it should rollback that txn
// **unconditionally**.
// In this case, TiKV set the lock TTL = 0, and TiDB use currentTS = 0 to call
// getTxnStatus, and getTxnStatus with currentTS = 0 would rollback the transaction.
if (lock.getTtl() == 0) {
return getTxnStatus(bo, lock.getTxnID(), lock.getPrimary(), 0L);
}
long currentTS = pdClient.getTimestamp(bo).getVersion();
return getTxnStatus(bo, lock.getTxnID(), lock.getPrimary(), currentTS);
}
private TxnStatus getTxnStatus(BackOffer bo, Long txnID, ByteString primary, Long currentTS) {
TxnStatus status = getResolved(txnID);
if (status != null) {
return status;
}
Supplier<CleanupRequest> factory =
() -> {
TiRegion primaryKeyRegion = regionManager.getRegionByKey(primary);
return CleanupRequest.newBuilder()
.setContext(primaryKeyRegion.getContext())
.setKey(primary)
.setStartVersion(txnID)
.setCurrentTs(currentTS)
.build();
};
status = new TxnStatus();
while (true) {
TiRegion primaryKeyRegion = regionManager.getRegionByKey(primary);
// new RegionStoreClient for PrimaryKey
RegionStoreClient primaryKeyRegionStoreClient = clientBuilder.build(primary);
KVErrorHandler<CleanupResponse> handler =
new KVErrorHandler<>(
regionManager,
primaryKeyRegionStoreClient,
primaryKeyRegionStoreClient.lockResolverClient,
primaryKeyRegion,
resp -> resp.hasRegionError() ? resp.getRegionError() : null,
resp -> resp.hasError() ? resp.getError() : null,
resolveLockResult -> null,
0L,
false);
CleanupResponse resp =
primaryKeyRegionStoreClient.callWithRetry(
bo, TikvGrpc.getKvCleanupMethod(), factory, handler);
if (resp == null) {
logger.error("getKvCleanupMethod failed without a cause");
regionManager.onRequestFail(primaryKeyRegion);
bo.doBackOff(
BoRegionMiss,
new TiClientInternalException("getKvCleanupMethod failed without a cause"));
continue;
}
if (resp.hasRegionError()) {
bo.doBackOff(BoRegionMiss, new RegionException(resp.getRegionError()));
continue;
}
if (resp.hasError()) {
Kvrpcpb.KeyError keyError = resp.getError();
// If the TTL of the primary lock is not outdated, the proto returns a ErrLocked contains
// the TTL.
if (keyError.hasLocked()) {
Kvrpcpb.LockInfo lockInfo = keyError.getLocked();
return new TxnStatus(lockInfo.getLockTtl(), 0L);
}
logger.error(String.format("unexpected cleanup err: %s, tid: %d", keyError, txnID));
throw new KeyException(keyError);
}
if (resp.getCommitVersion() != 0) {
status = new TxnStatus(0L, resp.getCommitVersion());
}
saveResolved(txnID, status);
return status;
}
}
private void saveResolved(long txnID, TxnStatus status) {
try {
readWriteLock.writeLock().lock();
if (resolved.containsKey(txnID)) {
return;
}
resolved.put(txnID, status);
recentResolved.add(txnID);
if (recentResolved.size() > RESOLVED_TXN_CACHE_SIZE) {
Long front = recentResolved.remove();
resolved.remove(front);
}
} finally {
readWriteLock.writeLock().unlock();
}
}
private TxnStatus getResolved(Long txnID) {
try {
readWriteLock.readLock().lock();
return resolved.get(txnID);
} finally {
readWriteLock.readLock().unlock();
}
}
}

View File

@ -0,0 +1,451 @@
/*
*
* Copyright 2020 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*
*/
package org.tikv.txn;
import static org.tikv.common.util.BackOffFunction.BackOffFuncType.BoRegionMiss;
import static org.tikv.common.util.BackOffFunction.BackOffFuncType.BoTxnNotFound;
import com.google.protobuf.ByteString;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Supplier;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.tikv.common.PDClient;
import org.tikv.common.TiConfiguration;
import org.tikv.common.exception.KeyException;
import org.tikv.common.exception.RegionException;
import org.tikv.common.exception.TiClientInternalException;
import org.tikv.common.operation.KVErrorHandler;
import org.tikv.common.region.AbstractRegionStoreClient;
import org.tikv.common.region.RegionManager;
import org.tikv.common.region.RegionStoreClient;
import org.tikv.common.region.TiRegion;
import org.tikv.common.region.TiRegion.RegionVerID;
import org.tikv.common.util.BackOffer;
import org.tikv.common.util.ChannelFactory;
import org.tikv.common.util.TsoUtils;
import org.tikv.kvproto.Kvrpcpb;
import org.tikv.kvproto.TikvGrpc;
import org.tikv.kvproto.TikvGrpc.TikvBlockingStub;
import org.tikv.kvproto.TikvGrpc.TikvStub;
import org.tikv.txn.exception.TxnNotFoundException;
import org.tikv.txn.exception.WriteConflictException;
/** Since v4.0.0 TiDB write will not block read (update MinCommitTS). */
public class LockResolverClientV4 extends AbstractRegionStoreClient
implements AbstractLockResolverClient {
private static final Logger logger = LoggerFactory.getLogger(LockResolverClientV4.class);
private final ReadWriteLock readWriteLock;
/**
* Note: Because the internal of long is same as unsigned_long and Txn id are never changed. Be
* careful to compare between two tso the `resolved` mapping is as {@code Map<TxnId, TxnStatus>}
* TxnStatus represents a txn's final status. It should be Commit or Rollback. if TxnStatus > 0,
* means the commit ts, otherwise abort
*/
private final Map<Long, TxnStatus> resolved;
/** the list is chain of txn for O(1) lru cache */
private final Queue<Long> recentResolved;
private final PDClient pdClient;
private final RegionStoreClient.RegionStoreClientBuilder clientBuilder;
public LockResolverClientV4(
TiConfiguration conf,
TiRegion region,
TikvBlockingStub blockingStub,
TikvStub asyncStub,
ChannelFactory channelFactory,
RegionManager regionManager,
PDClient pdClient,
RegionStoreClient.RegionStoreClientBuilder clientBuilder) {
super(conf, region, channelFactory, blockingStub, asyncStub, regionManager);
resolved = new HashMap<>();
recentResolved = new LinkedList<>();
readWriteLock = new ReentrantReadWriteLock();
this.pdClient = pdClient;
this.clientBuilder = clientBuilder;
}
@Override
public String getVersion() {
return "V4";
}
@Override
public ResolveLockResult resolveLocks(
BackOffer bo, long callerStartTS, List<Lock> locks, boolean forWrite) {
TxnExpireTime msBeforeTxnExpired = new TxnExpireTime();
if (locks.isEmpty()) {
return new ResolveLockResult(msBeforeTxnExpired.value());
}
Map<Long, Set<RegionVerID>> cleanTxns = new HashMap<>();
boolean pushFail = false;
List<Long> pushed = new ArrayList<>(locks.size());
for (Lock l : locks) {
TxnStatus status = getTxnStatusFromLock(bo, l, callerStartTS);
if (status.getTtl() == 0) {
Set<RegionVerID> cleanRegion =
cleanTxns.computeIfAbsent(l.getTxnID(), k -> new HashSet<>());
if (l.getLockType() == Kvrpcpb.Op.PessimisticLock) {
resolvePessimisticLock(bo, l, cleanRegion);
} else {
resolveLock(bo, l, status, cleanRegion);
}
} else {
long msBeforeLockExpired = TsoUtils.untilExpired(l.getTxnID(), status.getTtl());
msBeforeTxnExpired.update(msBeforeLockExpired);
if (forWrite) {
// Write conflict detected!
// If it's a optimistic conflict and current txn is earlier than the lock owner,
// abort current transaction.
// This could avoids the deadlock scene of two large transaction.
if (l.getLockType() != Kvrpcpb.Op.PessimisticLock && l.getTxnID() > callerStartTS) {
throw new WriteConflictException(
callerStartTS, l.getTxnID(), status.getCommitTS(), l.getKey().toByteArray());
}
} else {
if (status.getAction() != Kvrpcpb.Action.MinCommitTSPushed) {
pushFail = true;
} else {
pushed.add(l.getTxnID());
}
}
}
}
if (pushFail) {
pushed = new ArrayList<>();
}
return new ResolveLockResult(msBeforeTxnExpired.value(), pushed);
}
private void resolvePessimisticLock(BackOffer bo, Lock lock, Set<RegionVerID> cleanRegion) {
while (true) {
region = regionManager.getRegionByKey(lock.getKey());
if (cleanRegion.contains(region.getVerID())) {
return;
}
final long forUpdateTS =
lock.getLockForUpdateTs() == 0L ? Long.MAX_VALUE : lock.getLockForUpdateTs();
Supplier<Kvrpcpb.PessimisticRollbackRequest> factory =
() ->
Kvrpcpb.PessimisticRollbackRequest.newBuilder()
.setContext(region.getContext())
.setStartVersion(lock.getTxnID())
.setForUpdateTs(forUpdateTS)
.addKeys(lock.getKey())
.build();
KVErrorHandler<Kvrpcpb.PessimisticRollbackResponse> handler =
new KVErrorHandler<>(
regionManager,
this,
this,
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null,
resp -> resp.getErrorsCount() > 0 ? resp.getErrorsList().get(0) : null,
resolveLockResult -> null,
0L,
false);
Kvrpcpb.PessimisticRollbackResponse resp =
callWithRetry(bo, TikvGrpc.getKVPessimisticRollbackMethod(), factory, handler);
if (resp == null) {
logger.error("getKVPessimisticRollbackMethod failed without a cause");
regionManager.onRequestFail(region);
bo.doBackOff(
BoRegionMiss,
new TiClientInternalException("getKVPessimisticRollbackMethod failed without a cause"));
continue;
}
if (resp.hasRegionError()) {
bo.doBackOff(BoRegionMiss, new RegionException(resp.getRegionError()));
continue;
}
if (resp.getErrorsCount() > 0) {
logger.error(
String.format(
"unexpected resolveLock err: %s, lock: %s", resp.getErrorsList().get(0), lock));
throw new KeyException(resp.getErrorsList().get(0));
}
}
}
private TxnStatus getTxnStatusFromLock(BackOffer bo, Lock lock, long callerStartTS) {
long currentTS;
if (lock.getTtl() == 0) {
// NOTE: l.TTL = 0 is a special protocol!!!
// When the pessimistic txn prewrite meets locks of a txn, it should resolve the lock
// **unconditionally**.
// In this case, TiKV use lock TTL = 0 to notify TiDB, and TiDB should resolve the lock!
// Set currentTS to max uint64 to make the lock expired.
currentTS = Long.MAX_VALUE;
} else {
currentTS = pdClient.getTimestamp(bo).getVersion();
}
boolean rollbackIfNotExist = false;
while (true) {
try {
return getTxnStatus(
bo, lock.getTxnID(), lock.getPrimary(), callerStartTS, currentTS, rollbackIfNotExist);
} catch (TxnNotFoundException e) {
// If the error is something other than txnNotFoundErr, throw the error (network
// unavailable, tikv down, backoff timeout etc) to the caller.
logger.warn("getTxnStatus error!", e);
// Handle txnNotFound error.
// getTxnStatus() returns it when the secondary locks exist while the primary lock doesn't.
// This is likely to happen in the concurrently prewrite when secondary regions
// success before the primary region.
bo.doBackOff(BoTxnNotFound, e);
}
if (TsoUtils.untilExpired(lock.getTxnID(), lock.getTtl()) <= 0) {
logger.warn(
String.format(
"lock txn not found, lock has expired, CallerStartTs=%d lock str=%s",
callerStartTS, lock.toString()));
if (lock.getLockType() == Kvrpcpb.Op.PessimisticLock) {
return new TxnStatus();
}
rollbackIfNotExist = true;
} else {
if (lock.getLockType() == Kvrpcpb.Op.PessimisticLock) {
return new TxnStatus(lock.getTtl());
}
}
}
}
/**
* getTxnStatus sends the CheckTxnStatus request to the TiKV server. When rollbackIfNotExist is
* false, the caller should be careful with the TxnNotFoundException error.
*/
private TxnStatus getTxnStatus(
BackOffer bo,
Long txnID,
ByteString primary,
Long callerStartTS,
Long currentTS,
boolean rollbackIfNotExist) {
TxnStatus status = getResolved(txnID);
if (status != null) {
return status;
}
// CheckTxnStatus may meet the following cases:
// 1. LOCK
// 1.1 Lock expired -- orphan lock, fail to update TTL, crash recovery etc.
// 1.2 Lock TTL -- active transaction holding the lock.
// 2. NO LOCK
// 2.1 Txn Committed
// 2.2 Txn Rollbacked -- rollback itself, rollback by others, GC tomb etc.
// 2.3 No lock -- pessimistic lock rollback, concurrence prewrite.
Supplier<Kvrpcpb.CheckTxnStatusRequest> factory =
() -> {
TiRegion primaryKeyRegion = regionManager.getRegionByKey(primary);
return Kvrpcpb.CheckTxnStatusRequest.newBuilder()
.setContext(primaryKeyRegion.getContext())
.setPrimaryKey(primary)
.setLockTs(txnID)
.setCallerStartTs(callerStartTS)
.setCurrentTs(currentTS)
.setRollbackIfNotExist(rollbackIfNotExist)
.build();
};
while (true) {
TiRegion primaryKeyRegion = regionManager.getRegionByKey(primary);
// new RegionStoreClient for PrimaryKey
RegionStoreClient primaryKeyRegionStoreClient = clientBuilder.build(primary);
KVErrorHandler<Kvrpcpb.CheckTxnStatusResponse> handler =
new KVErrorHandler<>(
regionManager,
primaryKeyRegionStoreClient,
primaryKeyRegionStoreClient.lockResolverClient,
primaryKeyRegion,
resp -> resp.hasRegionError() ? resp.getRegionError() : null,
resp -> resp.hasError() ? resp.getError() : null,
resolveLockResult -> null,
callerStartTS,
false);
Kvrpcpb.CheckTxnStatusResponse resp =
primaryKeyRegionStoreClient.callWithRetry(
bo, TikvGrpc.getKvCheckTxnStatusMethod(), factory, handler);
if (resp == null) {
logger.error("getKvCheckTxnStatusMethod failed without a cause");
regionManager.onRequestFail(primaryKeyRegion);
bo.doBackOff(
BoRegionMiss,
new TiClientInternalException("getKvCheckTxnStatusMethod failed without a cause"));
continue;
}
if (resp.hasRegionError()) {
bo.doBackOff(BoRegionMiss, new RegionException(resp.getRegionError()));
continue;
}
if (resp.hasError()) {
Kvrpcpb.KeyError keyError = resp.getError();
if (keyError.hasTxnNotFound()) {
throw new TxnNotFoundException();
}
logger.error(String.format("unexpected cleanup err: %s, tid: %d", keyError, txnID));
throw new KeyException(keyError);
}
if (resp.getLockTtl() != 0) {
status = new TxnStatus(resp.getLockTtl(), 0L, resp.getAction());
} else {
status = new TxnStatus(0L, resp.getCommitVersion(), resp.getAction());
saveResolved(txnID, status);
}
return status;
}
}
private void resolveLock(
BackOffer bo, Lock lock, TxnStatus txnStatus, Set<RegionVerID> cleanRegion) {
boolean cleanWholeRegion = lock.getTxnSize() >= BIG_TXN_THRESHOLD;
while (true) {
region = regionManager.getRegionByKey(lock.getKey());
if (cleanRegion.contains(region.getVerID())) {
return;
}
Kvrpcpb.ResolveLockRequest.Builder builder =
Kvrpcpb.ResolveLockRequest.newBuilder()
.setContext(region.getContext())
.setStartVersion(lock.getTxnID());
if (txnStatus.isCommitted()) {
// txn is committed with commitTS txnStatus
builder.setCommitVersion(txnStatus.getCommitTS());
}
if (lock.getTxnSize() < BIG_TXN_THRESHOLD) {
// Only resolve specified keys when it is a small transaction,
// prevent from scanning the whole region in this case.
builder.addKeys(lock.getKey());
}
Supplier<Kvrpcpb.ResolveLockRequest> factory = builder::build;
KVErrorHandler<Kvrpcpb.ResolveLockResponse> handler =
new KVErrorHandler<>(
regionManager,
this,
this,
region,
resp -> resp.hasRegionError() ? resp.getRegionError() : null,
resp -> resp.hasError() ? resp.getError() : null,
resolveLockResult -> null,
0L,
false);
Kvrpcpb.ResolveLockResponse resp =
callWithRetry(bo, TikvGrpc.getKvResolveLockMethod(), factory, handler);
if (resp == null) {
logger.error("getKvResolveLockMethod failed without a cause");
regionManager.onRequestFail(region);
bo.doBackOff(
BoRegionMiss,
new TiClientInternalException("getKvResolveLockMethod failed without a cause"));
continue;
}
if (resp.hasRegionError()) {
bo.doBackOff(BoRegionMiss, new RegionException(resp.getRegionError()));
continue;
}
if (resp.hasError()) {
logger.error(
String.format("unexpected resolveLock err: %s, lock: %s", resp.getError(), lock));
throw new KeyException(resp.getError());
}
if (cleanWholeRegion) {
cleanRegion.add(region.getVerID());
}
return;
}
}
private void saveResolved(long txnID, TxnStatus status) {
try {
readWriteLock.writeLock().lock();
if (resolved.containsKey(txnID)) {
return;
}
resolved.put(txnID, status);
recentResolved.add(txnID);
if (recentResolved.size() > RESOLVED_TXN_CACHE_SIZE) {
Long front = recentResolved.remove();
resolved.remove(front);
}
} finally {
readWriteLock.writeLock().unlock();
}
}
private TxnStatus getResolved(Long txnID) {
try {
readWriteLock.readLock().lock();
return resolved.get(txnID);
} finally {
readWriteLock.readLock().unlock();
}
}
}

View File

@ -0,0 +1,44 @@
/*
*
* Copyright 2020 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*
*/
package org.tikv.txn;
import java.util.ArrayList;
import java.util.List;
public class ResolveLockResult {
private final long msBeforeTxnExpired;
private final List<Long> resolvedLocks;
public ResolveLockResult(long msBeforeTxnExpired) {
this.msBeforeTxnExpired = msBeforeTxnExpired;
this.resolvedLocks = new ArrayList<>();
}
public ResolveLockResult(long msBeforeTxnExpired, List<Long> resolvedLocks) {
this.msBeforeTxnExpired = msBeforeTxnExpired;
this.resolvedLocks = resolvedLocks;
}
public long getMsBeforeTxnExpired() {
return msBeforeTxnExpired;
}
public List<Long> getResolvedLocks() {
return resolvedLocks;
}
}

View File

@ -0,0 +1,55 @@
/*
*
* Copyright 2020 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*
*/
package org.tikv.txn;
public class TxnExpireTime {
private boolean initialized = false;
private long txnExpire = 0;
public TxnExpireTime() {}
public TxnExpireTime(boolean initialized, long txnExpire) {
this.initialized = initialized;
this.txnExpire = txnExpire;
}
public void update(long lockExpire) {
if (lockExpire < 0) {
lockExpire = 0;
}
if (!this.initialized) {
this.txnExpire = lockExpire;
this.initialized = true;
return;
}
if (lockExpire < this.txnExpire) {
this.txnExpire = lockExpire;
}
}
public long value() {
if (!this.initialized) {
return 0L;
} else {
return this.txnExpire;
}
}
}

View File

@ -1,5 +1,188 @@
/*
* Copyright 2019 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.tikv.txn;
public class TxnKVClient {
// TODO: To be done.
import com.google.common.collect.Lists;
import com.google.protobuf.ByteString;
import io.grpc.StatusRuntimeException;
import java.util.Arrays;
import java.util.List;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.tikv.common.ReadOnlyPDClient;
import org.tikv.common.TiConfiguration;
import org.tikv.common.exception.GrpcException;
import org.tikv.common.exception.KeyException;
import org.tikv.common.exception.RegionException;
import org.tikv.common.exception.TiClientInternalException;
import org.tikv.common.exception.TiKVException;
import org.tikv.common.meta.TiTimestamp;
import org.tikv.common.region.RegionManager;
import org.tikv.common.region.RegionStoreClient;
import org.tikv.common.region.TiRegion;
import org.tikv.common.util.BackOffFunction;
import org.tikv.common.util.BackOffer;
import org.tikv.common.util.ConcreteBackOffer;
import org.tikv.kvproto.Kvrpcpb;
import org.tikv.kvproto.Metapb;
/** KV client of transaction APIs for GET/PUT/DELETE/SCAN */
public class TxnKVClient implements AutoCloseable {
private static final Logger LOG = LoggerFactory.getLogger(TxnKVClient.class);
private final RegionStoreClient.RegionStoreClientBuilder clientBuilder;
private final TiConfiguration conf;
private final RegionManager regionManager;
private final ReadOnlyPDClient pdClient;
public TxnKVClient(
TiConfiguration conf,
RegionStoreClient.RegionStoreClientBuilder clientBuilder,
ReadOnlyPDClient pdClient) {
this.conf = conf;
this.clientBuilder = clientBuilder;
this.regionManager = clientBuilder.getRegionManager();
this.pdClient = pdClient;
}
public TiConfiguration getConf() {
return conf;
}
public RegionManager getRegionManager() {
return regionManager;
}
public TiTimestamp getTimestamp() {
BackOffer bo = ConcreteBackOffer.newTsoBackOff();
TiTimestamp timestamp = new TiTimestamp(0, 0);
try {
while (true) {
try {
timestamp = pdClient.getTimestamp(bo);
break;
} catch (final TiKVException e) {
// retry is exhausted
bo.doBackOff(BackOffFunction.BackOffFuncType.BoPDRPC, e);
}
}
} catch (GrpcException e1) {
LOG.error("Get tso from pd failed,", e1);
}
return timestamp;
}
/** when encountered region error,ErrBodyMissing, and other errors */
public ClientRPCResult prewrite(
BackOffer backOffer,
List<Kvrpcpb.Mutation> mutations,
ByteString primary,
long lockTTL,
long startTs,
TiRegion tiRegion,
Metapb.Store store) {
ClientRPCResult result = new ClientRPCResult(true, false, null);
// send request
RegionStoreClient client = clientBuilder.build(tiRegion, store);
try {
client.prewrite(backOffer, primary, mutations, startTs, lockTTL);
} catch (Exception e) {
result.setSuccess(false);
// mark retryable, region error, should retry prewrite again
result.setRetry(retryableException(e));
result.setException(e);
}
return result;
}
/** TXN Heart Beat: update primary key ttl */
public ClientRPCResult txnHeartBeat(
BackOffer backOffer,
ByteString primaryLock,
long startTs,
long ttl,
TiRegion tiRegion,
Metapb.Store store) {
ClientRPCResult result = new ClientRPCResult(true, false, null);
// send request
RegionStoreClient client = clientBuilder.build(tiRegion, store);
try {
client.txnHeartBeat(backOffer, primaryLock, startTs, ttl);
} catch (Exception e) {
result.setSuccess(false);
// mark retryable, region error, should retry heart beat again
result.setRetry(retryableException(e));
result.setException(e);
}
return result;
}
/**
* Commit request of 2pc, add backoff logic when encountered region error, ErrBodyMissing, and
* other errors
*
* @param backOffer
* @param keys
* @param startTs
* @param commitTs
* @param tiRegion
* @return
*/
public ClientRPCResult commit(
BackOffer backOffer,
ByteString[] keys,
long startTs,
long commitTs,
TiRegion tiRegion,
Metapb.Store store) {
ClientRPCResult result = new ClientRPCResult(true, false, null);
// send request
RegionStoreClient client = clientBuilder.build(tiRegion, store);
List<ByteString> byteList = Lists.newArrayList();
byteList.addAll(Arrays.asList(keys));
try {
client.commit(backOffer, byteList, startTs, commitTs);
} catch (Exception e) {
result.setSuccess(false);
// mark retryable, region error, should retry prewrite again
result.setRetry(retryableException(e));
result.setException(e);
}
return result;
}
// According to TiDB's implementation, when it comes to rpc error
// commit status remains undecided.
// If we fail to receive response for the request that commits primary key, it will be
// undetermined whether this
// transaction has been successfully committed.
// Under this circumstance, we can not declare the commit is complete (may lead to data lost),
// nor can we throw
// an error (may lead to the duplicated key error when upper level restarts the transaction).
// Currently the best
// solution is to populate this error and let upper layer drop the connection to the corresponding
// mysql client.
// TODO: check this logic to see are we satisfied?
private boolean retryableException(Exception e) {
return e instanceof TiClientInternalException
|| e instanceof KeyException
|| e instanceof RegionException
|| e instanceof StatusRuntimeException;
}
@Override
public void close() throws Exception {}
}

View File

@ -0,0 +1,85 @@
/*
*
* Copyright 2020 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*
*/
package org.tikv.txn;
import org.tikv.kvproto.Kvrpcpb;
/**
* ttl > 0: lock is not resolved
*
* <p>ttl = 0 && commitTS = 0: lock is deleted
*
* <p>ttl = 0 && commitTS > 0: lock is committed
*/
public class TxnStatus {
private long ttl;
private long commitTS;
private Kvrpcpb.Action action;
public TxnStatus() {
this.ttl = 0L;
this.commitTS = 0L;
this.action = Kvrpcpb.Action.UNRECOGNIZED;
}
public TxnStatus(long ttl) {
this.ttl = ttl;
this.commitTS = 0L;
this.action = Kvrpcpb.Action.UNRECOGNIZED;
}
public TxnStatus(long ttl, long commitTS) {
this.ttl = ttl;
this.commitTS = commitTS;
this.action = Kvrpcpb.Action.UNRECOGNIZED;
}
public TxnStatus(long ttl, long commitTS, Kvrpcpb.Action action) {
this.ttl = ttl;
this.commitTS = commitTS;
this.action = action;
}
public long getTtl() {
return ttl;
}
public void setTtl(long ttl) {
this.ttl = ttl;
}
public long getCommitTS() {
return commitTS;
}
public void setCommitTS(long commitTS) {
this.commitTS = commitTS;
}
public boolean isCommitted() {
return ttl == 0 && commitTS > 0;
}
public Kvrpcpb.Action getAction() {
return action;
}
public void setAction(Kvrpcpb.Action action) {
this.action = action;
}
}

View File

@ -0,0 +1,17 @@
/*
* Copyright 2020 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.tikv.txn.exception;
public class TxnNotFoundException extends RuntimeException {}

View File

@ -0,0 +1,27 @@
/*
* Copyright 2020 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.tikv.txn.exception;
import org.tikv.common.codec.KeyUtils;
public class WriteConflictException extends RuntimeException {
public WriteConflictException(long callerStartTS, long txnID, long commitTS, byte[] key) {
super(
String.format(
"callerStartTS=%d txnID=%d commitTS=%d key=%s",
callerStartTS, txnID, commitTS, KeyUtils.formatBytes(key)));
}
}

View File

@ -46,8 +46,8 @@ public class KVMockServer extends TikvGrpc.TikvImplBase {
private int port;
private Server server;
private TiRegion region;
private TreeMap<Key, ByteString> dataMap = new TreeMap<>();
private Map<ByteString, Integer> errorMap = new HashMap<>();
private final TreeMap<Key, ByteString> dataMap = new TreeMap<>();
private final Map<ByteString, Integer> errorMap = new HashMap<>();
// for KV error
public static final int ABORT = 1;
@ -303,7 +303,7 @@ public class KVMockServer extends TikvGrpc.TikvImplBase {
verifyContext(requestWrap.getContext());
DAGRequest request = DAGRequest.parseFrom(requestWrap.getData());
if (request.getStartTs() == 0) {
if (request.getStartTsFallback() == 0) {
throw new Exception();
}

View File

@ -1,33 +1,22 @@
package org.tikv.common;
import com.google.protobuf.ByteString;
import org.junit.After;
import java.io.IOException;
import org.junit.Before;
import org.tikv.common.TiConfiguration.KVMode;
import org.tikv.common.region.TiRegion;
import org.tikv.kvproto.Kvrpcpb;
import org.tikv.kvproto.Metapb;
import org.tikv.kvproto.Pdpb;
public class MockServerTest {
public class MockServerTest extends PDMockServerTest {
public KVMockServer server;
public PDMockServer pdServer;
public static final String LOCAL_ADDR = "127.0.0.1";
static final long CLUSTER_ID = 1024;
public int port;
public TiSession session;
public TiRegion region;
@Before
public void setUp() throws Exception {
pdServer = new PDMockServer();
pdServer.start(CLUSTER_ID);
pdServer.addGetMemberResp(
GrpcUtils.makeGetMembersResponse(
pdServer.getClusterId(),
GrpcUtils.makeMember(1, "http://" + LOCAL_ADDR + ":" + pdServer.port),
GrpcUtils.makeMember(2, "http://" + LOCAL_ADDR + ":" + (pdServer.port + 1)),
GrpcUtils.makeMember(2, "http://" + LOCAL_ADDR + ":" + (pdServer.port + 2))));
@Override
public void setUp() throws IOException {
super.setUp();
Metapb.Region r =
Metapb.Region.newBuilder()
@ -40,18 +29,13 @@ public class MockServerTest {
region =
new TiRegion(
r, r.getPeers(0), Kvrpcpb.IsolationLevel.RC, Kvrpcpb.CommandPri.Low, KVMode.TXN);
r,
r.getPeers(0),
session.getConf().getIsolationLevel(),
session.getConf().getCommandPriority(),
KVMode.TXN);
pdServer.addGetRegionResp(Pdpb.GetRegionResponse.newBuilder().setRegion(r).build());
server = new KVMockServer();
port = server.start(region);
// No PD needed in this test
TiConfiguration conf = TiConfiguration.createDefault("127.0.0.1:" + pdServer.port);
session = TiSession.create(conf);
}
@After
public void tearDown() throws Exception {
server.stop();
session.close();
}
}

View File

@ -20,10 +20,7 @@ import static org.tikv.common.GrpcUtils.encodeKey;
import com.google.common.collect.ImmutableList;
import com.google.protobuf.ByteString;
import java.io.IOException;
import java.util.concurrent.*;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.tikv.common.exception.GrpcException;
import org.tikv.common.meta.TiTimestamp;
@ -34,38 +31,13 @@ import org.tikv.kvproto.Metapb;
import org.tikv.kvproto.Metapb.Store;
import org.tikv.kvproto.Metapb.StoreState;
public class PDClientTest {
private PDMockServer server;
private static final long CLUSTER_ID = 1024;
private static final String LOCAL_ADDR = "127.0.0.1";
public class PDClientTest extends PDMockServerTest {
private static final String LOCAL_ADDR_IPV6 = "[::]";
private static TiSession session;
@Before
public void setup() throws IOException {
server = new PDMockServer();
server.start(CLUSTER_ID);
server.addGetMemberResp(
GrpcUtils.makeGetMembersResponse(
server.getClusterId(),
GrpcUtils.makeMember(1, "http://" + LOCAL_ADDR + ":" + server.port),
GrpcUtils.makeMember(2, "http://" + LOCAL_ADDR + ":" + (server.port + 1)),
GrpcUtils.makeMember(2, "http://" + LOCAL_ADDR + ":" + (server.port + 2))));
TiConfiguration conf = TiConfiguration.createDefault(LOCAL_ADDR + ":" + server.port);
session = TiSession.create(conf);
}
@After
public void tearDown() {
session.close();
server.stop();
}
@Test
public void testCreate() throws Exception {
try (PDClient client = session.getPDClient()) {
assertEquals(client.getLeaderWrapper().getLeaderInfo(), LOCAL_ADDR + ":" + server.port);
assertEquals(client.getLeaderWrapper().getLeaderInfo(), LOCAL_ADDR + ":" + pdServer.port);
assertEquals(client.getHeader().getClusterId(), CLUSTER_ID);
}
}
@ -73,24 +45,17 @@ public class PDClientTest {
@Test
public void testSwitchLeader() throws Exception {
try (PDClient client = session.getPDClient()) {
client.switchLeader(ImmutableList.of("http://" + LOCAL_ADDR + ":" + (server.port + 1)));
assertEquals(client.getLeaderWrapper().getLeaderInfo(), LOCAL_ADDR + ":" + (server.port + 1));
client.switchLeader(ImmutableList.of("http://" + LOCAL_ADDR + ":" + (pdServer.port + 1)));
assertEquals(
client.getLeaderWrapper().getLeaderInfo(), LOCAL_ADDR + ":" + (pdServer.port + 1));
}
tearDown();
server = new PDMockServer();
server.start(CLUSTER_ID);
server.addGetMemberResp(
GrpcUtils.makeGetMembersResponse(
server.getClusterId(),
GrpcUtils.makeMember(1, "http://" + LOCAL_ADDR_IPV6 + ":" + server.port),
GrpcUtils.makeMember(2, "http://" + LOCAL_ADDR_IPV6 + ":" + (server.port + 1)),
GrpcUtils.makeMember(2, "http://" + LOCAL_ADDR_IPV6 + ":" + (server.port + 2))));
TiConfiguration conf = TiConfiguration.createDefault(LOCAL_ADDR_IPV6 + ":" + server.port);
session = TiSession.create(conf);
setUp(LOCAL_ADDR_IPV6);
try (PDClient client = session.getPDClient()) {
client.switchLeader(ImmutableList.of("http://" + LOCAL_ADDR_IPV6 + ":" + (server.port + 2)));
client.switchLeader(
ImmutableList.of("http://" + LOCAL_ADDR_IPV6 + ":" + (pdServer.port + 2)));
assertEquals(
client.getLeaderWrapper().getLeaderInfo(), LOCAL_ADDR_IPV6 + ":" + (server.port + 2));
client.getLeaderWrapper().getLeaderInfo(), LOCAL_ADDR_IPV6 + ":" + (pdServer.port + 2));
}
}
@ -98,7 +63,7 @@ public class PDClientTest {
public void testTso() throws Exception {
try (PDClient client = session.getPDClient()) {
TiTimestamp ts = client.getTimestamp(defaultBackOff());
// Test server is set to generate physical == logical + 1
// Test pdServer is set to generate physical == logical + 1
assertEquals(ts.getPhysical(), ts.getLogical() + 1);
}
}
@ -109,9 +74,9 @@ public class PDClientTest {
byte[] endKey = new byte[] {1, 0, 2, 5};
int confVer = 1026;
int ver = 1027;
server.addGetRegionResp(
pdServer.addGetRegionResp(
GrpcUtils.makeGetRegionResponse(
server.getClusterId(),
pdServer.getClusterId(),
GrpcUtils.makeRegion(
1,
encodeKey(startKey),
@ -136,9 +101,9 @@ public class PDClientTest {
byte[] endKey = new byte[] {1, 0, 2, 5};
int confVer = 1026;
int ver = 1027;
server.addGetRegionResp(
pdServer.addGetRegionResp(
GrpcUtils.makeGetRegionResponse(
server.getClusterId(),
pdServer.getClusterId(),
GrpcUtils.makeRegion(
1,
encodeKey(startKey),
@ -164,9 +129,9 @@ public class PDClientTest {
int confVer = 1026;
int ver = 1027;
server.addGetRegionByIDResp(
pdServer.addGetRegionByIDResp(
GrpcUtils.makeGetRegionResponse(
server.getClusterId(),
pdServer.getClusterId(),
GrpcUtils.makeRegion(
1,
encodeKey(startKey),
@ -191,9 +156,9 @@ public class PDClientTest {
byte[] endKey = new byte[] {1, 0, 2, 5};
int confVer = 1026;
int ver = 1027;
server.addGetRegionByIDResp(
pdServer.addGetRegionByIDResp(
GrpcUtils.makeGetRegionResponse(
server.getClusterId(),
pdServer.getClusterId(),
GrpcUtils.makeRegion(
1,
encodeKey(startKey),
@ -216,9 +181,9 @@ public class PDClientTest {
public void testGetStore() throws Exception {
long storeId = 1;
String testAddress = "testAddress";
server.addGetStoreResp(
pdServer.addGetStoreResp(
GrpcUtils.makeGetStoreResponse(
server.getClusterId(),
pdServer.getClusterId(),
GrpcUtils.makeStore(
storeId,
testAddress,
@ -235,9 +200,9 @@ public class PDClientTest {
assertEquals(r.getLabels(0).getValue(), "v1");
assertEquals(r.getLabels(1).getValue(), "v2");
server.addGetStoreResp(
pdServer.addGetStoreResp(
GrpcUtils.makeGetStoreResponse(
server.getClusterId(),
pdServer.getClusterId(),
GrpcUtils.makeStore(storeId, testAddress, Metapb.StoreState.Tombstone)));
assertEquals(StoreState.Tombstone, client.getStore(defaultBackOff(), 0).getState());
}
@ -247,9 +212,9 @@ public class PDClientTest {
public void testGetStoreAsync() throws Exception {
long storeId = 1;
String testAddress = "testAddress";
server.addGetStoreResp(
pdServer.addGetStoreResp(
GrpcUtils.makeGetStoreResponse(
server.getClusterId(),
pdServer.getClusterId(),
GrpcUtils.makeStore(
storeId,
testAddress,
@ -266,9 +231,9 @@ public class PDClientTest {
assertEquals(r.getLabels(0).getValue(), "v1");
assertEquals(r.getLabels(1).getValue(), "v2");
server.addGetStoreResp(
pdServer.addGetStoreResp(
GrpcUtils.makeGetStoreResponse(
server.getClusterId(),
pdServer.getClusterId(),
GrpcUtils.makeStore(storeId, testAddress, Metapb.StoreState.Tombstone)));
assertEquals(
StoreState.Tombstone, client.getStoreAsync(defaultBackOff(), 0).get().getState());
@ -283,33 +248,33 @@ public class PDClientTest {
public void testRetryPolicy() throws Exception {
long storeId = 1024;
ExecutorService service = Executors.newCachedThreadPool();
server.addGetStoreResp(null);
server.addGetStoreResp(null);
server.addGetStoreResp(
pdServer.addGetStoreResp(null);
pdServer.addGetStoreResp(null);
pdServer.addGetStoreResp(
GrpcUtils.makeGetStoreResponse(
server.getClusterId(), GrpcUtils.makeStore(storeId, "", Metapb.StoreState.Up)));
pdServer.getClusterId(), GrpcUtils.makeStore(storeId, "", Metapb.StoreState.Up)));
try (PDClient client = session.getPDClient()) {
Callable<Store> storeCallable =
() -> client.getStore(ConcreteBackOffer.newCustomBackOff(5000), 0);
Future<Store> storeFuture = service.submit(storeCallable);
try {
Store r = storeFuture.get(5, TimeUnit.SECONDS);
Store r = storeFuture.get(50, TimeUnit.SECONDS);
assertEquals(r.getId(), storeId);
} catch (TimeoutException e) {
fail();
}
// Should fail
server.addGetStoreResp(null);
server.addGetStoreResp(null);
server.addGetStoreResp(null);
server.addGetStoreResp(null);
server.addGetStoreResp(null);
server.addGetStoreResp(null);
pdServer.addGetStoreResp(null);
pdServer.addGetStoreResp(null);
pdServer.addGetStoreResp(null);
pdServer.addGetStoreResp(null);
pdServer.addGetStoreResp(null);
pdServer.addGetStoreResp(null);
server.addGetStoreResp(
pdServer.addGetStoreResp(
GrpcUtils.makeGetStoreResponse(
server.getClusterId(), GrpcUtils.makeStore(storeId, "", Metapb.StoreState.Up)));
pdServer.getClusterId(), GrpcUtils.makeStore(storeId, "", Metapb.StoreState.Up)));
try {
client.getStore(defaultBackOff(), 0);
} catch (GrpcException e) {

View File

@ -0,0 +1,51 @@
/*
* Copyright 2020 PingCAP, Inc.
*
* Licensed 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,
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.tikv.common;
import java.io.IOException;
import org.junit.After;
import org.junit.Before;
public abstract class PDMockServerTest {
protected static final String LOCAL_ADDR = "127.0.0.1";
static final long CLUSTER_ID = 1024;
protected static TiSession session;
protected PDMockServer pdServer;
@Before
public void setUp() throws IOException {
setUp(LOCAL_ADDR);
}
void setUp(String addr) throws IOException {
pdServer = new PDMockServer();
pdServer.start(CLUSTER_ID);
pdServer.addGetMemberResp(
GrpcUtils.makeGetMembersResponse(
pdServer.getClusterId(),
GrpcUtils.makeMember(1, "http://" + addr + ":" + pdServer.port),
GrpcUtils.makeMember(2, "http://" + addr + ":" + (pdServer.port + 1)),
GrpcUtils.makeMember(3, "http://" + addr + ":" + (pdServer.port + 2))));
TiConfiguration conf = TiConfiguration.createDefault(addr + ":" + pdServer.port);
session = TiSession.create(conf);
}
@After
public void tearDown() throws Exception {
session.close();
pdServer.stop();
}
}

View File

@ -19,7 +19,6 @@ import static org.junit.Assert.*;
import com.google.protobuf.ByteString;
import java.io.IOException;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.tikv.common.region.RegionManager;
@ -29,33 +28,14 @@ import org.tikv.kvproto.Metapb;
import org.tikv.kvproto.Metapb.Store;
import org.tikv.kvproto.Metapb.StoreState;
public class RegionManagerTest {
private PDMockServer server;
private static final long CLUSTER_ID = 1024;
private static final String LOCAL_ADDR = "127.0.0.1";
public class RegionManagerTest extends PDMockServerTest {
private RegionManager mgr;
private TiSession session;
@Before
public void setup() throws IOException {
server = new PDMockServer();
server.start(CLUSTER_ID);
server.addGetMemberResp(
GrpcUtils.makeGetMembersResponse(
server.getClusterId(),
GrpcUtils.makeMember(1, "http://" + LOCAL_ADDR + ":" + server.port),
GrpcUtils.makeMember(2, "http://" + LOCAL_ADDR + ":" + (server.port + 1)),
GrpcUtils.makeMember(2, "http://" + LOCAL_ADDR + ":" + (server.port + 2))));
TiConfiguration conf = TiConfiguration.createDefault("127.0.0.1:" + server.port);
session = TiSession.create(conf);
mgr = new RegionManager(session.getPDClient());
}
@After
public void tearDown() {
server.stop();
session.close();
@Override
public void setUp() throws IOException {
super.setUp();
mgr = session.getRegionManager();
}
@Test
@ -67,9 +47,9 @@ public class RegionManagerTest {
int confVer = 1026;
int ver = 1027;
long regionId = 233;
server.addGetRegionResp(
pdServer.addGetRegionResp(
GrpcUtils.makeGetRegionResponse(
server.getClusterId(),
pdServer.getClusterId(),
GrpcUtils.makeRegion(
regionId,
GrpcUtils.encodeKey(startKey.toByteArray()),
@ -102,9 +82,9 @@ public class RegionManagerTest {
int confVer = 1026;
int ver = 1027;
long regionId = 233;
server.addGetRegionResp(
pdServer.addGetRegionResp(
GrpcUtils.makeGetRegionResponse(
server.getClusterId(),
pdServer.getClusterId(),
GrpcUtils.makeRegion(
regionId,
GrpcUtils.encodeKey(startKey.toByteArray()),
@ -112,9 +92,9 @@ public class RegionManagerTest {
GrpcUtils.makeRegionEpoch(confVer, ver),
GrpcUtils.makePeer(storeId, 10),
GrpcUtils.makePeer(storeId + 1, 20))));
server.addGetStoreResp(
pdServer.addGetStoreResp(
GrpcUtils.makeGetStoreResponse(
server.getClusterId(),
pdServer.getClusterId(),
GrpcUtils.makeStore(
storeId,
testAddress,
@ -126,48 +106,13 @@ public class RegionManagerTest {
assertEquals(pair.first.getId(), storeId);
}
@Test
public void getRegionById() throws Exception {
ByteString startKey = ByteString.copyFrom(new byte[] {1});
ByteString endKey = ByteString.copyFrom(new byte[] {10});
int confVer = 1026;
int ver = 1027;
long regionId = 233;
server.addGetRegionByIDResp(
GrpcUtils.makeGetRegionResponse(
server.getClusterId(),
GrpcUtils.makeRegion(
regionId,
GrpcUtils.encodeKey(startKey.toByteArray()),
GrpcUtils.encodeKey(endKey.toByteArray()),
GrpcUtils.makeRegionEpoch(confVer, ver),
GrpcUtils.makePeer(1, 10),
GrpcUtils.makePeer(2, 20))));
TiRegion region = mgr.getRegionById(regionId);
assertEquals(region.getId(), regionId);
TiRegion regionToSearch = mgr.getRegionById(regionId);
assertEquals(region, regionToSearch);
mgr.invalidateRegion(regionId);
// This will in turn invoke rpc and results in an error
// since we set just one rpc response
try {
mgr.getRegionById(regionId);
fail();
} catch (Exception ignored) {
}
}
@Test
public void getStoreById() throws Exception {
long storeId = 234;
String testAddress = "testAddress";
server.addGetStoreResp(
pdServer.addGetStoreResp(
GrpcUtils.makeGetStoreResponse(
server.getClusterId(),
pdServer.getClusterId(),
GrpcUtils.makeStore(
storeId,
testAddress,
@ -177,9 +122,9 @@ public class RegionManagerTest {
Store store = mgr.getStoreById(storeId);
assertEquals(store.getId(), storeId);
server.addGetStoreResp(
pdServer.addGetStoreResp(
GrpcUtils.makeGetStoreResponse(
server.getClusterId(),
pdServer.getClusterId(),
GrpcUtils.makeStore(
storeId + 1,
testAddress,

View File

@ -31,26 +31,39 @@ import org.tikv.kvproto.Metapb;
public class RegionStoreClientTest extends MockServerTest {
private RegionStoreClient createClient() {
private RegionStoreClient createClientV2() {
return createClient("2.1.19");
}
private RegionStoreClient createClientV3() {
return createClient("3.0.12");
}
private RegionStoreClient createClient(String version) {
Metapb.Store store =
Metapb.Store.newBuilder()
.setAddress(LOCAL_ADDR + ":" + port)
.setId(1)
.setState(Metapb.StoreState.Up)
.setVersion(version)
.build();
RegionStoreClientBuilder builder =
new RegionStoreClientBuilder(
session.getConf(),
session.getChannelFactory(),
new RegionManager(session.getPDClient()));
new RegionManager(session.getPDClient()),
session.getPDClient());
return builder.build(region, store);
}
@Test
public void rawGetTest() throws Exception {
RegionStoreClient client = createClient();
doRawGetTest(createClientV3());
}
public void doRawGetTest(RegionStoreClient client) throws Exception {
server.put("key1", "value1");
ByteString value = client.rawGet(defaultBackOff(), ByteString.copyFromUtf8("key1"));
assertEquals(ByteString.copyFromUtf8("value1"), value);
@ -74,7 +87,10 @@ public class RegionStoreClientTest extends MockServerTest {
@Test
public void getTest() throws Exception {
RegionStoreClient client = createClient();
doGetTest(createClientV3());
}
public void doGetTest(RegionStoreClient client) throws Exception {
server.put("key1", "value1");
ByteString value = client.get(defaultBackOff(), ByteString.copyFromUtf8("key1"), 1);
assertEquals(ByteString.copyFromUtf8("value1"), value);
@ -92,8 +108,10 @@ public class RegionStoreClientTest extends MockServerTest {
@Test
public void batchGetTest() throws Exception {
RegionStoreClient client = createClient();
doBatchGetTest(createClientV3());
}
public void doBatchGetTest(RegionStoreClient client) throws Exception {
server.put("key1", "value1");
server.put("key2", "value2");
server.put("key4", "value4");
@ -125,8 +143,10 @@ public class RegionStoreClientTest extends MockServerTest {
@Test
public void scanTest() throws Exception {
RegionStoreClient client = createClient();
doScanTest(createClientV3());
}
public void doScanTest(RegionStoreClient client) throws Exception {
server.put("key1", "value1");
server.put("key2", "value2");
server.put("key4", "value4");

View File

@ -6,10 +6,11 @@ import java.util.*;
import java.util.concurrent.*;
import java.util.stream.Collectors;
import org.apache.commons.lang3.RandomStringUtils;
import org.apache.log4j.Logger;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.tikv.common.TiConfiguration;
import org.tikv.common.TiSession;
import org.tikv.common.codec.KeyUtils;
@ -38,7 +39,7 @@ public class RawKVClientTest {
private static final ExecutorService executors = Executors.newFixedThreadPool(WORKER_CNT);
private final ExecutorCompletionService<Object> completionService =
new ExecutorCompletionService<>(executors);
private static final Logger logger = Logger.getLogger(RawKVClientTest.class);
private static final Logger logger = LoggerFactory.getLogger(RawKVClientTest.class);
private TiSession session;
static {