Simplified condition (#273)

Signed-off-by: Rahil <rahilrshk@gmail.com>
This commit is contained in:
Rahil 2021-09-22 08:22:18 +05:30 committed by GitHub
parent 0352c65931
commit 37506fe091
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
25 changed files with 117 additions and 123 deletions

View File

@ -1,14 +1,13 @@
package org.tikv.cdc; package org.tikv.cdc;
import java.util.Objects;
import org.tikv.kvproto.Cdcpb.Event.Row; import org.tikv.kvproto.Cdcpb.Event.Row;
class CDCEvent { class CDCEvent {
enum CDCEventType { enum CDCEventType {
ROW, ROW,
RESOLVED_TS, RESOLVED_TS,
ERROR; ERROR
}; }
public final long regionId; public final long regionId;
@ -57,7 +56,7 @@ class CDCEvent {
builder.append("resolvedTs=").append(resolvedTs); builder.append("resolvedTs=").append(resolvedTs);
break; break;
case ROW: case ROW:
builder.append("row=").append(Objects.toString(row)); builder.append("row=").append(row);
break; break;
} }
return builder.append("}").toString(); return builder.append("}").toString();

View File

@ -42,7 +42,7 @@ class RegionCDCClient implements AutoCloseable, StreamObserver<ChangeDataEvent>
private final AtomicBoolean running = new AtomicBoolean(false); private final AtomicBoolean running = new AtomicBoolean(false);
private boolean started = false; private final boolean started = false;
public RegionCDCClient( public RegionCDCClient(
final TiRegion region, final TiRegion region,

View File

@ -30,9 +30,8 @@ public class MetricsServer {
private static MetricsServer METRICS_SERVER_INSTANCE = null; private static MetricsServer METRICS_SERVER_INSTANCE = null;
private static int metricsServerRefCount = 0; private static int metricsServerRefCount = 0;
private int port; private final int port;
private HTTPServer server; private final HTTPServer server;
private CollectorRegistry collectorRegistry;
public static MetricsServer getInstance(TiConfiguration conf) { public static MetricsServer getInstance(TiConfiguration conf) {
if (!conf.isMetricsEnable()) { if (!conf.isMetricsEnable()) {
@ -58,16 +57,16 @@ public class MetricsServer {
private MetricsServer(int port) { private MetricsServer(int port) {
try { try {
this.collectorRegistry = new CollectorRegistry(); CollectorRegistry collectorRegistry = new CollectorRegistry();
this.collectorRegistry.register(RawKVClient.RAW_REQUEST_LATENCY); collectorRegistry.register(RawKVClient.RAW_REQUEST_LATENCY);
this.collectorRegistry.register(RawKVClient.RAW_REQUEST_FAILURE); collectorRegistry.register(RawKVClient.RAW_REQUEST_FAILURE);
this.collectorRegistry.register(RawKVClient.RAW_REQUEST_SUCCESS); collectorRegistry.register(RawKVClient.RAW_REQUEST_SUCCESS);
this.collectorRegistry.register(RegionStoreClient.GRPC_RAW_REQUEST_LATENCY); collectorRegistry.register(RegionStoreClient.GRPC_RAW_REQUEST_LATENCY);
this.collectorRegistry.register(RetryPolicy.GRPC_SINGLE_REQUEST_LATENCY); collectorRegistry.register(RetryPolicy.GRPC_SINGLE_REQUEST_LATENCY);
this.collectorRegistry.register(RegionManager.GET_REGION_BY_KEY_REQUEST_LATENCY); collectorRegistry.register(RegionManager.GET_REGION_BY_KEY_REQUEST_LATENCY);
this.collectorRegistry.register(PDClient.PD_GET_REGION_BY_KEY_REQUEST_LATENCY); collectorRegistry.register(PDClient.PD_GET_REGION_BY_KEY_REQUEST_LATENCY);
this.port = port; this.port = port;
this.server = new HTTPServer(new InetSocketAddress(port), this.collectorRegistry, true); this.server = new HTTPServer(new InetSocketAddress(port), collectorRegistry, true);
logger.info("http server is up " + this.server.getPort()); logger.info("http server is up " + this.server.getPort());
} catch (Exception e) { } catch (Exception e) {
logger.error("http server not up"); logger.error("http server not up");

View File

@ -188,7 +188,7 @@ public class PDClient extends AbstractGRPCClient<PDBlockingStub, PDStub>
"wait scatter region %d at key %s is %s", "wait scatter region %d at key %s is %s",
region.getId(), region.getId(),
KeyUtils.formatBytes(resp.getDesc().toByteArray()), KeyUtils.formatBytes(resp.getDesc().toByteArray()),
resp.getStatus().toString())); resp.getStatus()));
} }
} }
} }
@ -422,11 +422,9 @@ public class PDClient extends AbstractGRPCClient<PDBlockingStub, PDStub>
logger.info(String.format("can not switch to new leader, try follower forward")); logger.info(String.format("can not switch to new leader, try follower forward"));
List<Pdpb.Member> members = resp.getMembersList(); List<Pdpb.Member> members = resp.getMembersList();
boolean hasReachNextMember = false;
// If we have not used follower forward, try the first follower. // If we have not used follower forward, try the first follower.
if (pdClientWrapper != null && pdClientWrapper.getStoreAddress().equals(leaderUrlStr)) { boolean hasReachNextMember =
hasReachNextMember = true; pdClientWrapper != null && pdClientWrapper.getStoreAddress().equals(leaderUrlStr);
}
for (int i = 0; i < members.size() * 2; i++) { for (int i = 0; i < members.size() * 2; i++) {
Pdpb.Member member = members.get(i % members.size()); Pdpb.Member member = members.get(i % members.size());

View File

@ -266,7 +266,7 @@ public class TiConfiguration implements Serializable {
private boolean showRowId = getBoolean(TIKV_SHOW_ROWID); private boolean showRowId = getBoolean(TIKV_SHOW_ROWID);
private String dbPrefix = get(TIKV_DB_PREFIX); private String dbPrefix = get(TIKV_DB_PREFIX);
private KVMode kvMode = getKvMode(TIKV_KV_MODE); private KVMode kvMode = getKvMode(TIKV_KV_MODE);
private boolean enableGrpcForward = getBoolean(TIKV_ENABLE_GRPC_FORWARD); private final boolean enableGrpcForward = getBoolean(TIKV_ENABLE_GRPC_FORWARD);
private int kvClientConcurrency = getInt(TIKV_KV_CLIENT_CONCURRENCY); private int kvClientConcurrency = getInt(TIKV_KV_CLIENT_CONCURRENCY);
private ReplicaRead replicaRead = getReplicaRead(TIKV_REPLICA_READ); private ReplicaRead replicaRead = getReplicaRead(TIKV_REPLICA_READ);
@ -275,8 +275,8 @@ public class TiConfiguration implements Serializable {
private boolean metricsEnable = getBoolean(TIKV_METRICS_ENABLE); private boolean metricsEnable = getBoolean(TIKV_METRICS_ENABLE);
private int metricsPort = getInt(TIKV_METRICS_PORT); private int metricsPort = getInt(TIKV_METRICS_PORT);
private int grpcHealthCheckTimeout = getInt(TIKV_GRPC_HEALTH_CHECK_TIMEOUT); private final int grpcHealthCheckTimeout = getInt(TIKV_GRPC_HEALTH_CHECK_TIMEOUT);
private int healthCheckPeriodDuration = getInt(TIKV_HEALTH_CHECK_PERIOD_DURATION); private final int healthCheckPeriodDuration = getInt(TIKV_HEALTH_CHECK_PERIOD_DURATION);
private final String networkMappingName = get(TIKV_NETWORK_MAPPING_NAME); private final String networkMappingName = get(TIKV_NETWORK_MAPPING_NAME);
private HostMapping hostMapping = null; private HostMapping hostMapping = null;

View File

@ -73,7 +73,7 @@ public class TiSession implements AutoCloseable {
private volatile ImporterStoreClient.ImporterStoreClientBuilder importerClientBuilder; private volatile ImporterStoreClient.ImporterStoreClientBuilder importerClientBuilder;
private volatile boolean isClosed = false; private volatile boolean isClosed = false;
private volatile SwitchTiKVModeClient switchTiKVModeClient; private volatile SwitchTiKVModeClient switchTiKVModeClient;
private MetricsServer metricsServer; private final MetricsServer metricsServer;
private static final int MAX_SPLIT_REGION_STACK_DEPTH = 6; private static final int MAX_SPLIT_REGION_STACK_DEPTH = 6;
public TiSession(TiConfiguration conf) { public TiSession(TiConfiguration conf) {

View File

@ -79,7 +79,7 @@ public class RangeColumnPartitionPruner
NormalizedPredicate predicate = node.normalize(); NormalizedPredicate predicate = node.normalize();
if (predicate == null) { if (predicate == null) {
throw new UnsupportedOperationException( throw new UnsupportedOperationException(
String.format("ComparisonBinaryExpression %s cannot be normalized", node.toString())); String.format("ComparisonBinaryExpression %s cannot be normalized", node));
} }
String colRefName = predicate.getColumnRef().getName(); String colRefName = predicate.getColumnRef().getName();
List<Expression> partExprs = partExprsPerColumnRef.get(colRefName); List<Expression> partExprs = partExprsPerColumnRef.get(colRefName);

View File

@ -117,7 +117,7 @@ public class AstBuilder extends MySqlParserBaseVisitor<Expression> {
return parseIntOrLongOrDec(val); return parseIntOrLongOrDec(val);
} }
throw new UnsupportedSyntaxException(ctx.toString() + ": it is not supported."); throw new UnsupportedSyntaxException(ctx + ": it is not supported.");
} }
@Override @Override
@ -135,7 +135,7 @@ public class AstBuilder extends MySqlParserBaseVisitor<Expression> {
} }
return Constant.create(sb.toString().replace("\"", "")); return Constant.create(sb.toString().replace("\"", ""));
} }
throw new UnsupportedSyntaxException(ctx.toString() + " is not supported yet"); throw new UnsupportedSyntaxException(ctx + " is not supported yet");
} }
@Override @Override
@ -161,7 +161,7 @@ public class AstBuilder extends MySqlParserBaseVisitor<Expression> {
Doubles.tryParse(ctx.REAL_LITERAL().getSymbol().getText()), RealType.REAL); Doubles.tryParse(ctx.REAL_LITERAL().getSymbol().getText()), RealType.REAL);
} }
throw new UnsupportedSyntaxException(ctx.toString() + "not supported constant"); throw new UnsupportedSyntaxException(ctx + "not supported constant");
} }
@Override @Override
@ -187,8 +187,7 @@ public class AstBuilder extends MySqlParserBaseVisitor<Expression> {
return ComparisonBinaryExpression.greaterEqual(left, right); return ComparisonBinaryExpression.greaterEqual(left, right);
} }
throw new UnsupportedSyntaxException( throw new UnsupportedSyntaxException(ctx + ": it is not possible reach to this line of code");
ctx.toString() + ": it is not possible reach to this line of code");
} }
public Expression visitLogicalExpression(MySqlParser.LogicalExpressionContext ctx) { public Expression visitLogicalExpression(MySqlParser.LogicalExpressionContext ctx) {
@ -203,8 +202,7 @@ public class AstBuilder extends MySqlParserBaseVisitor<Expression> {
return LogicalBinaryExpression.xor(visitChildren(left), visitChildren(right)); return LogicalBinaryExpression.xor(visitChildren(left), visitChildren(right));
} }
throw new UnsupportedSyntaxException( throw new UnsupportedSyntaxException(ctx + ": it is not possible reach to this line of code");
ctx.toString() + ": it is not possible reach to this line of code");
} }
@Override @Override
@ -222,6 +220,6 @@ public class AstBuilder extends MySqlParserBaseVisitor<Expression> {
case "div": case "div":
return ArithmeticBinaryExpression.divide(left, right); return ArithmeticBinaryExpression.divide(left, right);
} }
throw new UnsupportedSyntaxException(ctx.toString() + ": it is not supported right now"); throw new UnsupportedSyntaxException(ctx + ": it is not supported right now");
} }
} }

View File

@ -41,9 +41,9 @@ public abstract class RetryPolicy<RespT> {
.register(); .register();
// handles PD and TiKV's error. // handles PD and TiKV's error.
private ErrorHandler<RespT> handler; private final ErrorHandler<RespT> handler;
private ImmutableSet<Status.Code> unrecoverableStatus = private final ImmutableSet<Status.Code> unrecoverableStatus =
ImmutableSet.of( ImmutableSet.of(
Status.Code.ALREADY_EXISTS, Status.Code.PERMISSION_DENIED, Status.Code.ALREADY_EXISTS, Status.Code.PERMISSION_DENIED,
Status.Code.INVALID_ARGUMENT, Status.Code.NOT_FOUND, Status.Code.INVALID_ARGUMENT, Status.Code.NOT_FOUND,

View File

@ -151,7 +151,7 @@ public class RegionManager {
ByteString key, TiStoreType storeType, BackOffer backOffer) { ByteString key, TiStoreType storeType, BackOffer backOffer) {
TiRegion region = getRegionByKey(key, backOffer); TiRegion region = getRegionByKey(key, backOffer);
if (!region.isValid()) { if (!region.isValid()) {
throw new TiClientInternalException("Region invalid: " + region.toString()); throw new TiClientInternalException("Region invalid: " + region);
} }
TiStore store = null; TiStore store = null;
@ -181,7 +181,7 @@ public class RegionManager {
if (store == null) { if (store == null) {
throw new TiClientInternalException( throw new TiClientInternalException(
"Cannot find valid store on " + storeType + " for region " + region.toString()); "Cannot find valid store on " + storeType + " for region " + region);
} }
return Pair.create(region, store); return Pair.create(region, store);

View File

@ -19,12 +19,12 @@ import org.tikv.kvproto.Metapb;
public class StoreHealthyChecker implements Runnable { public class StoreHealthyChecker implements Runnable {
private static final Logger logger = LoggerFactory.getLogger(StoreHealthyChecker.class); private static final Logger logger = LoggerFactory.getLogger(StoreHealthyChecker.class);
private static final long MAX_CHECK_STORE_TOMBSTONE_TICK = 60; private static final long MAX_CHECK_STORE_TOMBSTONE_TICK = 60;
private BlockingQueue<TiStore> taskQueue; private final BlockingQueue<TiStore> taskQueue;
private final ChannelFactory channelFactory; private final ChannelFactory channelFactory;
private final ReadOnlyPDClient pdClient; private final ReadOnlyPDClient pdClient;
private final RegionCache cache; private final RegionCache cache;
private long checkTombstoneTick; private long checkTombstoneTick;
private long timeout; private final long timeout;
public StoreHealthyChecker( public StoreHealthyChecker(
ChannelFactory channelFactory, ReadOnlyPDClient pdClient, RegionCache cache, long timeout) { ChannelFactory channelFactory, ReadOnlyPDClient pdClient, RegionCache cache, long timeout) {
@ -37,11 +37,8 @@ public class StoreHealthyChecker implements Runnable {
} }
public boolean scheduleStoreHealthCheck(TiStore store) { public boolean scheduleStoreHealthCheck(TiStore store) {
if (!this.taskQueue.add(store)) {
// add queue false, mark it reachable so that it can be put again. // add queue false, mark it reachable so that it can be put again.
return false; return this.taskQueue.add(store);
}
return true;
} }
private List<TiStore> getValidStores() { private List<TiStore> getValidStores() {
@ -68,11 +65,7 @@ public class StoreHealthyChecker implements Runnable {
HealthGrpc.newBlockingStub(channel).withDeadlineAfter(timeout, TimeUnit.MILLISECONDS); HealthGrpc.newBlockingStub(channel).withDeadlineAfter(timeout, TimeUnit.MILLISECONDS);
HealthCheckRequest req = HealthCheckRequest.newBuilder().build(); HealthCheckRequest req = HealthCheckRequest.newBuilder().build();
HealthCheckResponse resp = stub.check(req); HealthCheckResponse resp = stub.check(req);
if (resp.getStatus() == HealthCheckResponse.ServingStatus.SERVING) { return resp.getStatus() == HealthCheckResponse.ServingStatus.SERVING;
return true;
} else {
return false;
}
} catch (Exception e) { } catch (Exception e) {
return false; return false;
} }

View File

@ -6,13 +6,13 @@ import java.util.concurrent.atomic.AtomicLong;
import org.tikv.kvproto.Metapb; import org.tikv.kvproto.Metapb;
public class TiStore implements Serializable { public class TiStore implements Serializable {
private static long MAX_FAIL_FORWARD_TIMES = 4; private static final long MAX_FAIL_FORWARD_TIMES = 4;
private final Metapb.Store store; private final Metapb.Store store;
private final Metapb.Store proxyStore; private final Metapb.Store proxyStore;
private AtomicBoolean reachable; private final AtomicBoolean reachable;
private AtomicBoolean valid; private final AtomicBoolean valid;
private AtomicLong failForwardCount; private final AtomicLong failForwardCount;
private AtomicBoolean canForward; private final AtomicBoolean canForward;
public TiStore(Metapb.Store store) { public TiStore(Metapb.Store store) {
this.store = store; this.store = store;

View File

@ -25,8 +25,8 @@ import org.tikv.kvproto.Coprocessor;
import org.tikv.kvproto.Errorpb; import org.tikv.kvproto.Errorpb;
public class StreamingResponse implements Iterable { public class StreamingResponse implements Iterable {
private Iterator<Coprocessor.Response> resultIterator; private final Iterator<Coprocessor.Response> resultIterator;
private List<Coprocessor.Response> responseList; private final List<Coprocessor.Response> responseList;
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
public StreamingResponse(Iterator resultIterator) { public StreamingResponse(Iterator resultIterator) {

View File

@ -1097,9 +1097,9 @@ public class RawKVClient implements AutoCloseable {
private Iterator<KvPair> iterator; private Iterator<KvPair> iterator;
private ByteString startKey; private final ByteString startKey;
private ByteString endKey; private final ByteString endKey;
private boolean keyOnly; private final boolean keyOnly;
private KvPair last; private KvPair last;

View File

@ -244,7 +244,7 @@ public class LockResolverClientV4 extends AbstractRegionStoreClient
logger.warn( logger.warn(
String.format( String.format(
"lock txn not found, lock has expired, CallerStartTs=%d lock str=%s", "lock txn not found, lock has expired, CallerStartTs=%d lock str=%s",
callerStartTS, lock.toString())); callerStartTS, lock));
if (lock.getLockType() == Kvrpcpb.Op.PessimisticLock) { if (lock.getLockType() == Kvrpcpb.Op.PessimisticLock) {
return new TxnStatus(); return new TxnStatus();
} }

View File

@ -15,11 +15,18 @@
package org.tikv.common; package org.tikv.common;
import static org.junit.Assert.*; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.tikv.common.GrpcUtils.encodeKey; import static org.tikv.common.GrpcUtils.encodeKey;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
import java.util.concurrent.*; import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.junit.Test; import org.junit.Test;
import org.tikv.common.exception.GrpcException; import org.tikv.common.exception.GrpcException;
import org.tikv.common.meta.TiTimestamp; import org.tikv.common.meta.TiTimestamp;
@ -31,31 +38,33 @@ import org.tikv.kvproto.Metapb.Store;
import org.tikv.kvproto.Metapb.StoreState; import org.tikv.kvproto.Metapb.StoreState;
public class PDClientTest extends PDMockServerTest { public class PDClientTest extends PDMockServerTest {
private static final String LOCAL_ADDR_IPV6 = "[::]"; private static final String LOCAL_ADDR_IPV6 = "[::]";
public static final String HTTP = "http://";
@Test @Test
public void testCreate() throws Exception { public void testCreate() throws Exception {
try (PDClient client = session.getPDClient()) { try (PDClient client = session.getPDClient()) {
assertEquals(client.getPdClientWrapper().getLeaderInfo(), LOCAL_ADDR + ":" + pdServer.port); assertEquals(LOCAL_ADDR + ":" + pdServer.port, client.getPdClientWrapper().getLeaderInfo());
assertEquals(client.getHeader().getClusterId(), CLUSTER_ID); assertEquals(CLUSTER_ID, client.getHeader().getClusterId());
} }
} }
@Test @Test
public void testSwitchLeader() throws Exception { public void testSwitchLeader() throws Exception {
try (PDClient client = session.getPDClient()) { try (PDClient client = session.getPDClient()) {
client.trySwitchLeader("http://" + LOCAL_ADDR + ":" + (pdServer.port + 1)); client.trySwitchLeader(HTTP + LOCAL_ADDR + ":" + (pdServer.port + 1));
assertEquals( assertEquals(
client.getPdClientWrapper().getLeaderInfo(), client.getPdClientWrapper().getLeaderInfo(),
"http://" + LOCAL_ADDR + ":" + (pdServer.port + 1)); HTTP + LOCAL_ADDR + ":" + (pdServer.port + 1));
} }
tearDown(); tearDown();
setUp(LOCAL_ADDR_IPV6); setUp(LOCAL_ADDR_IPV6);
try (PDClient client = session.getPDClient()) { try (PDClient client = session.getPDClient()) {
client.trySwitchLeader("http://" + LOCAL_ADDR_IPV6 + ":" + (pdServer.port + 2)); client.trySwitchLeader(HTTP + LOCAL_ADDR_IPV6 + ":" + (pdServer.port + 2));
assertEquals( assertEquals(
client.getPdClientWrapper().getLeaderInfo(), client.getPdClientWrapper().getLeaderInfo(),
"http://" + LOCAL_ADDR_IPV6 + ":" + (pdServer.port + 2)); HTTP + LOCAL_ADDR_IPV6 + ":" + (pdServer.port + 2));
} }
} }
@ -93,8 +102,8 @@ public class PDClientTest extends PDMockServerTest {
assertEquals(r.getEndKey(), ByteString.copyFrom(endKey)); assertEquals(r.getEndKey(), ByteString.copyFrom(endKey));
assertEquals(r.getRegionEpoch().getConfVer(), confVer); assertEquals(r.getRegionEpoch().getConfVer(), confVer);
assertEquals(r.getRegionEpoch().getVersion(), ver); assertEquals(r.getRegionEpoch().getVersion(), ver);
assertEquals(l.getId(), 1); assertEquals(1, l.getId());
assertEquals(l.getStoreId(), 10); assertEquals(10, l.getStoreId());
} }
} }
@ -119,12 +128,12 @@ public class PDClientTest extends PDMockServerTest {
Pair<Metapb.Region, Metapb.Peer> rl = client.getRegionByID(defaultBackOff(), 0); Pair<Metapb.Region, Metapb.Peer> rl = client.getRegionByID(defaultBackOff(), 0);
Metapb.Region r = rl.first; Metapb.Region r = rl.first;
Metapb.Peer l = rl.second; Metapb.Peer l = rl.second;
assertEquals(r.getStartKey(), ByteString.copyFrom(startKey)); assertEquals(ByteString.copyFrom(startKey), r.getStartKey());
assertEquals(r.getEndKey(), ByteString.copyFrom(endKey)); assertEquals(ByteString.copyFrom(endKey), r.getEndKey());
assertEquals(r.getRegionEpoch().getConfVer(), confVer); assertEquals(confVer, r.getRegionEpoch().getConfVer());
assertEquals(r.getRegionEpoch().getVersion(), ver); assertEquals(ver, r.getRegionEpoch().getVersion());
assertEquals(l.getId(), 1); assertEquals(1, l.getId());
assertEquals(l.getStoreId(), 10); assertEquals(10, l.getStoreId());
} }
} }
@ -142,20 +151,20 @@ public class PDClientTest extends PDMockServerTest {
GrpcUtils.makeStoreLabel("k1", "v1"), GrpcUtils.makeStoreLabel("k1", "v1"),
GrpcUtils.makeStoreLabel("k2", "v2")))); GrpcUtils.makeStoreLabel("k2", "v2"))));
try (PDClient client = session.getPDClient()) { try (PDClient client = session.getPDClient()) {
Store r = client.getStore(defaultBackOff(), 0); Store r = client.getStore(defaultBackOff(), storeId);
assertEquals(r.getId(), storeId); assertEquals(storeId, r.getId());
assertEquals(r.getAddress(), testAddress); assertEquals(testAddress, r.getAddress());
assertEquals(r.getState(), Metapb.StoreState.Up); assertEquals(Metapb.StoreState.Up, r.getState());
assertEquals(r.getLabels(0).getKey(), "k1"); assertEquals("k1", r.getLabels(0).getKey());
assertEquals(r.getLabels(1).getKey(), "k2"); assertEquals("k2", r.getLabels(1).getKey());
assertEquals(r.getLabels(0).getValue(), "v1"); assertEquals("v1", r.getLabels(0).getValue());
assertEquals(r.getLabels(1).getValue(), "v2"); assertEquals("v2", r.getLabels(1).getValue());
pdServer.addGetStoreResp( pdServer.addGetStoreResp(
GrpcUtils.makeGetStoreResponse( GrpcUtils.makeGetStoreResponse(
pdServer.getClusterId(), pdServer.getClusterId(),
GrpcUtils.makeStore(storeId, testAddress, Metapb.StoreState.Tombstone))); GrpcUtils.makeStore(storeId, testAddress, Metapb.StoreState.Tombstone)));
assertEquals(StoreState.Tombstone, client.getStore(defaultBackOff(), 0).getState()); assertEquals(StoreState.Tombstone, client.getStore(defaultBackOff(), storeId).getState());
} }
} }

View File

@ -15,7 +15,9 @@
package org.tikv.common; package org.tikv.common;
import static org.junit.Assert.*; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.fail;
import com.google.common.collect.RangeMap; import com.google.common.collect.RangeMap;
import com.google.common.collect.TreeRangeMap; import com.google.common.collect.TreeRangeMap;
@ -53,7 +55,7 @@ public class RegionManagerTest extends PDMockServerTest {
} }
@Test @Test
public void getRegionByKey() throws Exception { public void getRegionByKey() {
ByteString startKey = ByteString.copyFrom(new byte[] {1}); ByteString startKey = ByteString.copyFrom(new byte[] {1});
ByteString endKey = ByteString.copyFrom(new byte[] {10}); ByteString endKey = ByteString.copyFrom(new byte[] {10});
ByteString searchKey = ByteString.copyFrom(new byte[] {5}); ByteString searchKey = ByteString.copyFrom(new byte[] {5});
@ -96,7 +98,7 @@ public class RegionManagerTest extends PDMockServerTest {
} }
@Test @Test
public void getStoreByKey() throws Exception { public void getStoreByKey() {
ByteString startKey = ByteString.copyFrom(new byte[] {1}); ByteString startKey = ByteString.copyFrom(new byte[] {1});
ByteString endKey = ByteString.copyFrom(new byte[] {10}); ByteString endKey = ByteString.copyFrom(new byte[] {10});
ByteString searchKey = ByteString.copyFrom(new byte[] {5}); ByteString searchKey = ByteString.copyFrom(new byte[] {5});
@ -133,7 +135,7 @@ public class RegionManagerTest extends PDMockServerTest {
} }
@Test @Test
public void getStoreById() throws Exception { public void getStoreById() {
long storeId = 234; long storeId = 234;
String testAddress = "testAddress"; String testAddress = "testAddress";
pdServer.addGetStoreResp( pdServer.addGetStoreResp(

View File

@ -15,7 +15,9 @@
package org.tikv.common; package org.tikv.common;
import static org.junit.Assert.*; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.protobuf.ByteString; import com.google.protobuf.ByteString;
@ -62,11 +64,11 @@ public class RegionStoreClientTest extends MockServerTest {
} }
@Test @Test
public void rawGetTest() throws Exception { public void rawGetTest() {
doRawGetTest(createClientV3()); doRawGetTest(createClientV3());
} }
public void doRawGetTest(RegionStoreClient client) throws Exception { public void doRawGetTest(RegionStoreClient client) {
server.put("key1", "value1"); server.put("key1", "value1");
Optional<ByteString> value = client.rawGet(defaultBackOff(), ByteString.copyFromUtf8("key1")); Optional<ByteString> value = client.rawGet(defaultBackOff(), ByteString.copyFromUtf8("key1"));
assertEquals(ByteString.copyFromUtf8("value1"), value.get()); assertEquals(ByteString.copyFromUtf8("value1"), value.get());
@ -93,7 +95,7 @@ public class RegionStoreClientTest extends MockServerTest {
doGetTest(createClientV3()); doGetTest(createClientV3());
} }
public void doGetTest(RegionStoreClient client) throws Exception { public void doGetTest(RegionStoreClient client) {
server.put("key1", "value1"); server.put("key1", "value1");
ByteString value = client.get(defaultBackOff(), ByteString.copyFromUtf8("key1"), 1); ByteString value = client.get(defaultBackOff(), ByteString.copyFromUtf8("key1"), 1);
assertEquals(ByteString.copyFromUtf8("value1"), value); assertEquals(ByteString.copyFromUtf8("value1"), value);
@ -110,11 +112,11 @@ public class RegionStoreClientTest extends MockServerTest {
} }
@Test @Test
public void batchGetTest() throws Exception { public void batchGetTest() {
doBatchGetTest(createClientV3()); doBatchGetTest(createClientV3());
} }
public void doBatchGetTest(RegionStoreClient client) throws Exception { public void doBatchGetTest(RegionStoreClient client) {
server.put("key1", "value1"); server.put("key1", "value1");
server.put("key2", "value2"); server.put("key2", "value2");
server.put("key4", "value4"); server.put("key4", "value4");
@ -145,11 +147,11 @@ public class RegionStoreClientTest extends MockServerTest {
} }
@Test @Test
public void scanTest() throws Exception { public void scanTest() {
doScanTest(createClientV3()); doScanTest(createClientV3());
} }
public void doScanTest(RegionStoreClient client) throws Exception { public void doScanTest(RegionStoreClient client) {
server.put("key1", "value1"); server.put("key1", "value1");
server.put("key2", "value2"); server.put("key2", "value2");
server.put("key4", "value4"); server.put("key4", "value4");

View File

@ -23,7 +23,7 @@ import org.tikv.common.codec.Codec.*;
public class CodecTest { public class CodecTest {
@Test @Test
public void readNWriteLongTest() throws Exception { public void readNWriteLongTest() {
CodecDataOutput cdo = new CodecDataOutput(); CodecDataOutput cdo = new CodecDataOutput();
IntegerCodec.writeLongFully(cdo, 9999L, true); IntegerCodec.writeLongFully(cdo, 9999L, true);
IntegerCodec.writeLongFully(cdo, -2333L, false); IntegerCodec.writeLongFully(cdo, -2333L, false);
@ -62,7 +62,7 @@ public class CodecTest {
} }
@Test @Test
public void readNWriteUnsignedLongTest() throws Exception { public void readNWriteUnsignedLongTest() {
CodecDataOutput cdo = new CodecDataOutput(); CodecDataOutput cdo = new CodecDataOutput();
IntegerCodec.writeULongFully(cdo, 0xffffffffffffffffL, true); IntegerCodec.writeULongFully(cdo, 0xffffffffffffffffL, true);
IntegerCodec.writeULongFully(cdo, Long.MIN_VALUE, false); IntegerCodec.writeULongFully(cdo, Long.MIN_VALUE, false);
@ -156,7 +156,7 @@ public class CodecTest {
} }
@Test @Test
public void writeBytesTest() throws Exception { public void writeBytesTest() {
CodecDataOutput cdo = new CodecDataOutput(); CodecDataOutput cdo = new CodecDataOutput();
Codec.BytesCodec.writeBytes(cdo, "abcdefghijk".getBytes()); Codec.BytesCodec.writeBytes(cdo, "abcdefghijk".getBytes());
byte[] result = cdo.toBytes(); byte[] result = cdo.toBytes();

View File

@ -57,17 +57,17 @@ public class MyDecimalTest {
@Test @Test
public void readWordTest() { public void readWordTest() {
assertEquals(MyDecimal.readWord(new int[] {250}, 1, 0), -6); assertEquals(-6, MyDecimal.readWord(new int[] {250}, 1, 0));
assertEquals(MyDecimal.readWord(new int[] {50}, 1, 0), 50); assertEquals(50, MyDecimal.readWord(new int[] {50}, 1, 0));
assertEquals(MyDecimal.readWord(new int[] {250, 250}, 2, 0), -1286); assertEquals(-1286, MyDecimal.readWord(new int[] {250, 250}, 2, 0));
assertEquals(MyDecimal.readWord(new int[] {50, 50}, 2, 0), 12850); assertEquals(12850, MyDecimal.readWord(new int[] {50, 50}, 2, 0));
assertEquals(MyDecimal.readWord(new int[] {250, 250, 250}, 3, 0), -328966); assertEquals(-328966, MyDecimal.readWord(new int[] {250, 250, 250}, 3, 0));
assertEquals(MyDecimal.readWord(new int[] {50, 50, 50}, 3, 0), 3289650); assertEquals(3289650, MyDecimal.readWord(new int[] {50, 50, 50}, 3, 0));
assertEquals(MyDecimal.readWord(new int[] {250, 250, 250, 250}, 4, 0), -84215046); assertEquals(-84215046, MyDecimal.readWord(new int[] {250, 250, 250, 250}, 4, 0));
assertEquals(MyDecimal.readWord(new int[] {50, 50, 50, 50}, 4, 0), 842150450); assertEquals(842150450, MyDecimal.readWord(new int[] {50, 50, 50, 50}, 4, 0));
} }
@Test @Test
@ -113,11 +113,6 @@ public class MyDecimalTest {
int precision; int precision;
int frac; int frac;
MyDecimalTestStruct(String in, String out) {
this.in = in;
this.out = out;
}
MyDecimalTestStruct(String in, String out, int precision, int frac) { MyDecimalTestStruct(String in, String out, int precision, int frac) {
this.in = in; this.in = in;
this.out = out; this.out = out;

View File

@ -24,7 +24,7 @@ public class SwitchTiKVModeTest {
} }
@Test @Test
public void swithTiKVModeTest() throws InterruptedException { public void switchTiKVModeTest() throws InterruptedException {
SwitchTiKVModeClient switchTiKVModeClient = session.getSwitchTiKVModeClient(); SwitchTiKVModeClient switchTiKVModeClient = session.getSwitchTiKVModeClient();
switchTiKVModeClient.keepTiKVToImportMode(); switchTiKVModeClient.keepTiKVToImportMode();
Thread.sleep(6000); Thread.sleep(6000);

View File

@ -38,7 +38,7 @@ public class TxnKVIngestTest {
} }
@Ignore @Ignore
public void txnIngestTest() throws InterruptedException { public void txnIngestTest() {
KVClient client = session.createKVClient(); KVClient client = session.createKVClient();
// gen test data // gen test data

View File

@ -24,7 +24,7 @@ import org.junit.Test;
public class KeyTest { public class KeyTest {
@Test @Test
public void nextPrefixTest() throws Exception { public void nextPrefixTest() {
Key k1 = toRawKey(new byte[] {1, 2, 3}); Key k1 = toRawKey(new byte[] {1, 2, 3});
assertEquals(toRawKey(new byte[] {1, 2, 4}), k1.nextPrefix()); assertEquals(toRawKey(new byte[] {1, 2, 4}), k1.nextPrefix());

View File

@ -13,7 +13,7 @@ import org.tikv.common.TiConfiguration;
import org.tikv.common.TiSession; import org.tikv.common.TiSession;
public class MetricsTest { public class MetricsTest {
private List<TiSession> sessionList = new ArrayList<>(); private final List<TiSession> sessionList = new ArrayList<>();
@After @After
public void tearDown() throws Exception { public void tearDown() throws Exception {
@ -35,7 +35,7 @@ public class MetricsTest {
client.put(ByteString.copyFromUtf8("k"), ByteString.copyFromUtf8("v")); client.put(ByteString.copyFromUtf8("k"), ByteString.copyFromUtf8("v"));
Optional<ByteString> result = client.get(ByteString.copyFromUtf8("k")); Optional<ByteString> result = client.get(ByteString.copyFromUtf8("k"));
assertTrue(result.isPresent()); assertTrue(result.isPresent());
assertEquals(result.get().toStringUtf8(), "v"); assertEquals("v", result.get().toStringUtf8());
client.close(); client.close();
session.close(); session.close();
} }
@ -61,7 +61,7 @@ public class MetricsTest {
Optional<ByteString> result = client2.get(ByteString.copyFromUtf8("k2")); Optional<ByteString> result = client2.get(ByteString.copyFromUtf8("k2"));
assertTrue(result.isPresent()); assertTrue(result.isPresent());
assertEquals(result.get().toStringUtf8(), "v2"); assertEquals("v2", result.get().toStringUtf8());
client2.close(); client2.close();
session2.close(); session2.close();
@ -82,7 +82,6 @@ public class MetricsTest {
conf2.setMetricsPort(54321); conf2.setMetricsPort(54321);
try { try {
TiSession.create(conf2); TiSession.create(conf2);
assertEquals(1, 2);
} catch (IllegalArgumentException e) { } catch (IllegalArgumentException e) {
assertEquals( assertEquals(
"Do dot support multiple tikv.metrics.port, which are 54321 and 12345", e.getMessage()); "Do dot support multiple tikv.metrics.port, which are 54321 and 12345", e.getMessage());

View File

@ -22,7 +22,7 @@ public class ReflectionWrapper {
this.obj = obj; this.obj = obj;
} }
private Object obj; private final Object obj;
public Object call(String methodName, Object... args) { public Object call(String methodName, Object... args) {
try { try {