diff --git a/src/main/java/org/tikv/BUILD b/src/main/java/org/tikv/BUILD
deleted file mode 100644
index e369326b877..00000000000
--- a/src/main/java/org/tikv/BUILD
+++ /dev/null
@@ -1,34 +0,0 @@
-package(default_visibility = ["//visibility:public"])
-
-java_library(
- name = "tikv-java-client-lib",
- srcs = glob(
- ["**/*.java"],
- ),
- deps = [
- "//:java",
- "@com_fasterxml_jackson_core_jackson_annotations//jar",
- "@com_fasterxml_jackson_core_jackson_core//jar",
- "@com_fasterxml_jackson_core_jackson_databind//jar",
- "@com_google_code_findbugs_jsr305//jar",
- "@com_google_code_gson_gson//jar",
- "@com_google_errorprone_error_prone_annotations//jar",
- "@com_google_guava_guava//jar",
- "@com_google_protobuf_protobuf_java//jar",
- "@joda_time//jar",
- # the following are defined in rules_protobuf
- "@org_pubref_rules_protobuf//java:grpc_compiletime_deps",
- "@org_pubref_rules_protobuf//java:netty_runtime_deps",
-
- "@org_slf4j_slf4j_api//jar",
- "@org_slf4j_jcl_over_slf4j//jar",
- "@org_slf4j_jul_to_slf4j//jar",
- "@log4j_log4j//jar",
- "@net_sf_trove4j_trove4j//jar",
- ],
-)
-
-filegroup(
- name = "srcs",
- srcs = ["BUILD"] + glob(["**/*.java"]),
-)
diff --git a/src/main/java/org/tikv/Main.java b/src/main/java/org/tikv/Main.java
deleted file mode 100644
index 82e18d9a6aa..00000000000
--- a/src/main/java/org/tikv/Main.java
+++ /dev/null
@@ -1,5 +0,0 @@
-package org.tikv;
-
-public class Main {
- public static void main(String args[]) throws Exception {}
-}
diff --git a/src/main/java/org/tikv/common/allocator/RowIDAllocator.java b/src/main/java/org/tikv/common/allocator/RowIDAllocator.java
deleted file mode 100644
index f7ca507215e..00000000000
--- a/src/main/java/org/tikv/common/allocator/RowIDAllocator.java
+++ /dev/null
@@ -1,258 +0,0 @@
-/*
- * 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.allocator;
-
-import com.google.common.primitives.UnsignedLongs;
-import com.google.protobuf.ByteString;
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.function.Function;
-import org.tikv.common.Snapshot;
-import org.tikv.common.TiSession;
-import org.tikv.common.codec.CodecDataInput;
-import org.tikv.common.codec.CodecDataOutput;
-import org.tikv.common.codec.MetaCodec;
-import org.tikv.common.exception.AllocateRowIDOverflowException;
-import org.tikv.common.exception.TiBatchWriteException;
-import org.tikv.common.meta.TiTableInfo;
-import org.tikv.common.util.BackOffer;
-import org.tikv.common.util.ConcreteBackOffer;
-import org.tikv.txn.TwoPhaseCommitter;
-
-/**
- * RowIDAllocator read current start from TiKV and write back 'start+step' back to TiKV. It designs
- * to allocate all id for data to be written at once, hence it does not need run inside a txn.
- *
- *
(start, end] is allocated
- */
-public final class RowIDAllocator implements Serializable {
- private final long maxShardRowIDBits;
- private final long dbId;
- private final TiSession session;
- private final long step;
- private long end;
-
- private RowIDAllocator(long maxShardRowIDBits, long dbId, long step, TiSession session) {
- this.maxShardRowIDBits = maxShardRowIDBits;
- this.dbId = dbId;
- this.step = step;
- this.session = session;
- }
-
- /**
- * @param index should >= 1
- * @return
- */
- public long getShardRowId(long index) {
- return getShardRowId(maxShardRowIDBits, index, index + getStart());
- }
-
- static long getShardRowId(long maxShardRowIDBits, long partitionIndex, long rowID) {
- if (maxShardRowIDBits <= 0 || maxShardRowIDBits >= 16) {
- return rowID;
- }
-
- // assert rowID < Math.pow(2, 64 - maxShardRowIDBits)
-
- long partition = partitionIndex & ((1L << maxShardRowIDBits) - 1);
- return rowID | (partition << (64 - maxShardRowIDBits - 1));
- }
-
- public static RowIDAllocator create(
- long dbId, TiTableInfo table, TiSession session, boolean unsigned, long step) {
- RowIDAllocator allocator =
- new RowIDAllocator(table.getMaxShardRowIDBits(), dbId, step, session);
- if (unsigned) {
- allocator.initUnsigned(session.createSnapshot(), table.getId(), table.getMaxShardRowIDBits());
- } else {
- allocator.initSigned(session.createSnapshot(), table.getId(), table.getMaxShardRowIDBits());
- }
-
- return allocator;
- }
-
- public long getStart() {
- return end - step;
- }
-
- public long getEnd() {
- return end;
- }
-
- // set key value pair to tikv via two phase committer protocol.
- private void set(ByteString key, byte[] value) {
- TwoPhaseCommitter twoPhaseCommitter =
- new TwoPhaseCommitter(session, session.getTimestamp().getVersion());
-
- twoPhaseCommitter.prewritePrimaryKey(
- ConcreteBackOffer.newCustomBackOff(BackOffer.PREWRITE_MAX_BACKOFF),
- key.toByteArray(),
- value);
-
- twoPhaseCommitter.commitPrimaryKey(
- ConcreteBackOffer.newCustomBackOff(BackOffer.BATCH_COMMIT_BACKOFF),
- key.toByteArray(),
- session.getTimestamp().getVersion());
-
- try {
- twoPhaseCommitter.close();
- } catch (Throwable ignored) {
- }
- }
-
- private void updateMeta(ByteString key, byte[] oldVal, Snapshot snapshot) {
- // 1. encode hash meta key
- // 2. load meta via hash meta key from TiKV
- // 3. update meta's filed count and set it back to TiKV
- CodecDataOutput cdo = new CodecDataOutput();
- ByteString metaKey = MetaCodec.encodeHashMetaKey(cdo, key.toByteArray());
- long fieldCount;
- ByteString metaVal = snapshot.get(metaKey);
-
- // decode long from bytes
- // big endian the 8 bytes
- fieldCount = new CodecDataInput(metaVal.toByteArray()).readLong();
-
- // update meta field count only oldVal is null
- if (oldVal == null || oldVal.length == 0) {
- fieldCount++;
- cdo.reset();
- cdo.writeLong(fieldCount);
-
- set(metaKey, cdo.toBytes());
- }
- }
-
- private long updateHash(
- ByteString key,
- ByteString field,
- Function calculateNewVal,
- Snapshot snapshot) {
- // 1. encode hash data key
- // 2. get value in byte from get operation
- // 3. calculate new value via calculateNewVal
- // 4. check old value equals to new value or not
- // 5. set the new value back to TiKV via 2pc
- // 6. encode a hash meta key
- // 7. update a hash meta field count if needed
-
- CodecDataOutput cdo = new CodecDataOutput();
- MetaCodec.encodeHashDataKey(cdo, key.toByteArray(), field.toByteArray());
- ByteString dataKey = cdo.toByteString();
- byte[] oldVal = snapshot.get(dataKey.toByteArray());
-
- byte[] newVal = calculateNewVal.apply(oldVal);
- if (Arrays.equals(newVal, oldVal)) {
- // not need to update
- return 0L;
- }
-
- set(dataKey, newVal);
- updateMeta(key, oldVal, snapshot);
- return Long.parseLong(new String(newVal));
- }
-
- private static boolean isDBExisted(long dbId, Snapshot snapshot) {
- ByteString dbKey = MetaCodec.encodeDatabaseID(dbId);
- ByteString json = MetaCodec.hashGet(MetaCodec.KEY_DBs, dbKey, snapshot);
- return json != null && !json.isEmpty();
- }
-
- private static boolean isTableExisted(long dbId, long tableId, Snapshot snapshot) {
- ByteString dbKey = MetaCodec.encodeDatabaseID(dbId);
- ByteString tableKey = MetaCodec.tableKey(tableId);
- return !MetaCodec.hashGet(dbKey, tableKey, snapshot).isEmpty();
- }
-
- public static boolean shardRowBitsOverflow(
- long base, long step, long shardRowBits, boolean reservedSignBit) {
- long signBit = reservedSignBit ? 1 : 0;
- long mask = ((1L << shardRowBits) - 1) << (64 - shardRowBits - signBit);
- if (reservedSignBit) {
- return ((base + step) & mask) > 0;
- } else {
- return Long.compareUnsigned((base + step) & mask, 0) > 0;
- }
- }
-
- /**
- * read current row id from TiKV and write the calculated value back to TiKV. The calculation rule
- * is start(read from TiKV) + step.
- */
- public long udpateAllocateId(
- long dbId, long tableId, long step, Snapshot snapshot, long shard, boolean hasSignedBit) {
- if (isDBExisted(dbId, snapshot) && isTableExisted(dbId, tableId, snapshot)) {
- return updateHash(
- MetaCodec.encodeDatabaseID(dbId),
- MetaCodec.autoTableIDKey(tableId),
- (oldVal) -> {
- long base = 0;
- if (oldVal != null && oldVal.length != 0) {
- base = Long.parseLong(new String(oldVal));
- }
- if (shard >= 1 && shardRowBitsOverflow(base, step, shard, hasSignedBit)) {
- throw new AllocateRowIDOverflowException(base, step, shard);
- }
- base += step;
- return String.valueOf(base).getBytes();
- },
- snapshot);
- }
-
- throw new IllegalArgumentException("table or database is not existed");
- }
-
- /** read current row id from TiKV according to database id and table id. */
- public static long getAllocateId(long dbId, long tableId, Snapshot snapshot) {
- if (isDBExisted(dbId, snapshot) && isTableExisted(dbId, tableId, snapshot)) {
- ByteString dbKey = MetaCodec.encodeDatabaseID(dbId);
- ByteString tblKey = MetaCodec.autoTableIDKey(tableId);
- ByteString val = MetaCodec.hashGet(dbKey, tblKey, snapshot);
- if (val.isEmpty()) return 0L;
- return Long.parseLong(val.toStringUtf8());
- }
-
- throw new IllegalArgumentException("table or database is not existed");
- }
-
- private void initSigned(Snapshot snapshot, long tableId, long shard) {
- // get new start from TiKV, and calculate new end and set it back to TiKV.
- long newStart = getAllocateId(dbId, tableId, snapshot);
- long tmpStep = Math.min(Long.MAX_VALUE - newStart, step);
- if (tmpStep != step) {
- throw new TiBatchWriteException("cannot allocate ids for this write");
- }
- if (newStart == Long.MAX_VALUE) {
- throw new TiBatchWriteException("cannot allocate more ids since it ");
- }
- end = udpateAllocateId(dbId, tableId, tmpStep, snapshot, shard, true);
- }
-
- private void initUnsigned(Snapshot snapshot, long tableId, long shard) {
- // get new start from TiKV, and calculate new end and set it back to TiKV.
- long newStart = getAllocateId(dbId, tableId, snapshot);
- // for unsigned long, -1L is max value.
- long tmpStep = UnsignedLongs.min(-1L - newStart, step);
- if (tmpStep != step) {
- throw new TiBatchWriteException("cannot allocate ids for this write");
- }
- // when compare unsigned long, the min value is largest value.
- if (UnsignedLongs.compare(newStart, -1L) == 0) {
- throw new TiBatchWriteException(
- "cannot allocate more ids since the start reaches " + "unsigned long's max value ");
- }
- end = udpateAllocateId(dbId, tableId, tmpStep, snapshot, shard, false);
- }
-}
diff --git a/src/main/java/org/tikv/common/operation/iterator/CoprocessorIterator.java b/src/main/java/org/tikv/common/operation/iterator/CoprocessorIterator.java
index 4870ee46252..431cd18b2e0 100644
--- a/src/main/java/org/tikv/common/operation/iterator/CoprocessorIterator.java
+++ b/src/main/java/org/tikv/common/operation/iterator/CoprocessorIterator.java
@@ -27,7 +27,11 @@
import org.tikv.common.TiSession;
import org.tikv.common.codec.Codec.IntegerCodec;
import org.tikv.common.codec.CodecDataInput;
-import org.tikv.common.columnar.*;
+import org.tikv.common.columnar.BatchedTiChunkColumnVector;
+import org.tikv.common.columnar.TiChunk;
+import org.tikv.common.columnar.TiChunkColumnVector;
+import org.tikv.common.columnar.TiColumnVector;
+import org.tikv.common.columnar.TiRowColumnVector;
import org.tikv.common.columnar.datatypes.CHType;
import org.tikv.common.meta.TiDAGRequest;
import org.tikv.common.operation.SchemaInfer;
diff --git a/src/main/java/org/tikv/txn/BatchKeys.java b/src/main/java/org/tikv/txn/BatchKeys.java
deleted file mode 100644
index bee0eb6a72e..00000000000
--- a/src/main/java/org/tikv/txn/BatchKeys.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * 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 keys;
- private final int sizeInBytes;
-
- public BatchKeys(
- TiRegion region, Metapb.Store store, List keysInput, int sizeInBytes) {
- this.region = region;
- this.store = store;
- this.keys = new ArrayList<>();
- this.keys.addAll(keysInput);
- this.sizeInBytes = sizeInBytes;
- }
-
- public List getKeys() {
- return keys;
- }
-
- public void setKeys(List 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;
- }
-}
diff --git a/src/main/java/org/tikv/txn/ClientRPCResult.java b/src/main/java/org/tikv/txn/ClientRPCResult.java
deleted file mode 100644
index 657f04fc885..00000000000
--- a/src/main/java/org/tikv/txn/ClientRPCResult.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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;
- }
-}
diff --git a/src/main/java/org/tikv/txn/GroupKeyResult.java b/src/main/java/org/tikv/txn/GroupKeyResult.java
deleted file mode 100644
index a7da770ecfa..00000000000
--- a/src/main/java/org/tikv/txn/GroupKeyResult.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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, List> groupsResult;
-
- public GroupKeyResult() {
- this.groupsResult = new HashMap<>();
- }
-
- public Map, List> getGroupsResult() {
- return groupsResult;
- }
-
- public void setGroupsResult(Map, List> groupsResult) {
- this.groupsResult = groupsResult;
- }
-}
diff --git a/src/test/java/org/tikv/BUILD b/src/test/java/org/tikv/BUILD
deleted file mode 100644
index b4c5d7d97ba..00000000000
--- a/src/test/java/org/tikv/BUILD
+++ /dev/null
@@ -1,24 +0,0 @@
-package(default_visibility = ["//visibility:public"])
-
-load(":rule.bzl", "junit_suite_test")
-
-junit_suite_test(
- name = "tikv-client-java-test",
- srcs = glob(
- ["**/*.java"],
- ),
- deps = [
- "//src/main/java/com/pingcap/tikv:tikv-java-client-lib",
- "//:java",
- "//:java_compile_imports",
- "@com_fasterxml_jackson_core_jackson_annotations//jar",
- "@com_fasterxml_jackson_core_jackson_core//jar",
- "@com_fasterxml_jackson_core_jackson_databind//jar",
-
- "@org_pubref_rules_protobuf//java:grpc_compiletime_deps",
- "@org_pubref_rules_protobuf//java:netty_runtime_deps",
- "@net_sf_trove4j_trove4j//jar",
- "@junit_junit//jar",
- "@joda_time//jar",
- ],
-)
diff --git a/src/test/java/org/tikv/rule.bzl b/src/test/java/org/tikv/rule.bzl
deleted file mode 100644
index 958c16c368a..00000000000
--- a/src/test/java/org/tikv/rule.bzl
+++ /dev/null
@@ -1,40 +0,0 @@
-def junit_suite_test(name, srcs, deps, size="small", resources=[], classpath_resources=[], jvm_flags=[], tags=[], data=[]):
- tests = []
- package = PACKAGE_NAME.replace("src/test/java/", "").replace("/", ".")
- for src in srcs:
- if src.endswith("Test.java"):
- if "/" in src:
- src = package + "." + src.replace("/", ".")
- tests += [src.replace(".java", ".class")]
-
-
- native.genrule(
- name = name + "-AllTests-gen",
- outs = ["AllTests.java"],
- cmd = """
- cat <> $@
-package %s;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
-@RunWith(Suite.class)
-@Suite.SuiteClasses({%s})
-public class AllTests {}
-EOF
- """ % (package, ",".join(tests))
- )
-
- native.java_test(
- name = name,
- srcs = srcs + ["AllTests.java"],
- test_class = package + ".AllTests",
- resources = resources,
- classpath_resources = classpath_resources,
- data = data,
- size = size,
- tags = tags,
- jvm_flags = jvm_flags,
- deps = deps + [
- ],
- )