Skip to content

Commit

Permalink
[improvement](cloud) Accelerate creating table by batching RPC
Browse files Browse the repository at this point in the history
fix

fix feut

fix

fix

fix fe

merge rpc
  • Loading branch information
deardeng committed Jun 27, 2024
1 parent 89265cf commit 7d4ce60
Show file tree
Hide file tree
Showing 20 changed files with 437 additions and 47 deletions.
2 changes: 2 additions & 0 deletions cloud/src/common/bvars.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -79,6 +79,8 @@ BvarLatencyRecorderWithTag g_bvar_ms_finish_tablet_job("ms", "finish_tablet_job"
BvarLatencyRecorderWithTag g_bvar_ms_get_cluster_status("ms", "get_cluster_status");
BvarLatencyRecorderWithTag g_bvar_ms_set_cluster_status("ms", "set_cluster_status");

BvarLatencyRecorderWithTag g_bvar_ms_check_kv("ms", "check_kv");

// txn_kv's bvars
bvar::LatencyRecorder g_bvar_txn_kv_get("txn_kv", "get");
bvar::LatencyRecorder g_bvar_txn_kv_range_get("txn_kv", "range_get");
Expand Down
1 change: 1 addition & 0 deletions cloud/src/common/bvars.h
Original file line number Diff line number Diff line change
Expand Up @@ -174,6 +174,7 @@ extern BvarLatencyRecorderWithTag g_bvar_ms_set_cluster_status;
extern BvarLatencyRecorderWithTag g_bvar_ms_get_instance;
extern BvarLatencyRecorderWithTag g_bvar_ms_get_rl_task_commit_attach;
extern BvarLatencyRecorderWithTag g_bvar_ms_get_txn_id;
extern BvarLatencyRecorderWithTag g_bvar_ms_check_kv;

// txn_kv's bvars
extern bvar::LatencyRecorder g_bvar_txn_kv_get;
Expand Down
15 changes: 15 additions & 0 deletions cloud/src/meta-service/meta_service.h
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,9 @@ class MetaServiceImpl : public cloud::MetaService {
void drop_index(::google::protobuf::RpcController* controller, const IndexRequest* request,
IndexResponse* response, ::google::protobuf::Closure* done) override;

void check_kv(::google::protobuf::RpcController* controller, const CheckKVRequest* request,
CheckKVResponse* response, ::google::protobuf::Closure* done) override;

void prepare_partition(::google::protobuf::RpcController* controller,
const PartitionRequest* request, PartitionResponse* response,
::google::protobuf::Closure* done) override;
Expand Down Expand Up @@ -280,6 +283,13 @@ class MetaServiceImpl : public cloud::MetaService {
const AlterInstanceRequest* request,
std::function<std::pair<MetaServiceCode, std::string>(InstanceInfoPB*)> action);

using check_create_table_type = std::function<const std::tuple<
const ::google::protobuf::RepeatedField<int64_t>, std::string,
std::function<std::string(std::string, int64_t)>>(const CheckKVRequest* request)>;
void check_create_table(std::string instance_id, const CheckKVRequest* request,
CheckKVResponse* response, MetaServiceCode* code, std::string* msg,
check_create_table_type get_check_info);

std::shared_ptr<TxnKv> txn_kv_;
std::shared_ptr<ResourceManager> resource_mgr_;
std::shared_ptr<RateLimiter> rate_limiter_;
Expand Down Expand Up @@ -426,6 +436,11 @@ class MetaServiceProxy final : public MetaService {
call_impl(&cloud::MetaService::drop_index, controller, request, response, done);
}

void check_kv(::google::protobuf::RpcController* controller, const CheckKVRequest* request,
CheckKVResponse* response, ::google::protobuf::Closure* done) override {
call_impl(&cloud::MetaService::check_kv, controller, request, response, done);
}

void prepare_partition(::google::protobuf::RpcController* controller,
const PartitionRequest* request, PartitionResponse* response,
::google::protobuf::Closure* done) override {
Expand Down
91 changes: 91 additions & 0 deletions cloud/src/meta-service/meta_service_partition.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,11 @@ static TxnErrorCode index_exists(Transaction* txn, const std::string& instance_i
return it->has_next() ? TxnErrorCode::TXN_OK : TxnErrorCode::TXN_KEY_NOT_FOUND;
}

static TxnErrorCode check_recycle_key_exist(Transaction* txn, const std::string& key) {
std::string val;
return txn->get(key, &val);
}

void MetaServiceImpl::prepare_index(::google::protobuf::RpcController* controller,
const IndexRequest* request, IndexResponse* response,
::google::protobuf::Closure* done) {
Expand Down Expand Up @@ -614,4 +619,90 @@ void MetaServiceImpl::drop_partition(::google::protobuf::RpcController* controll
}
}

void MetaServiceImpl::check_create_table(std::string instance_id, const CheckKVRequest* request,
CheckKVResponse* response, MetaServiceCode* code,
std::string* msg, check_create_table_type get_check_info) {
std::unique_ptr<Transaction> txn;
TxnErrorCode err = txn_kv_->create_txn(&txn);
if (err != TxnErrorCode::TXN_OK) {
*code = cast_as<ErrCategory::READ>(err);
*msg = "failed to create txn";
return;
}
auto& [keys, hint, key_func] = get_check_info(request);

if (keys.empty()) {
*code = MetaServiceCode::INVALID_ARGUMENT;
*msg = "empty partition_ids";
return;
}

for (auto id : keys) {
auto key = key_func(instance_id, id);
err = check_recycle_key_exist(txn.get(), key);
if (err == TxnErrorCode::TXN_KEY_NOT_FOUND) {
continue;
} else if (err == TxnErrorCode::TXN_OK) {
// find not match, prepare commit
*code = MetaServiceCode::INVALID_ARGUMENT;
*msg = "prepare and commit rpc not match, recycle key remained";
return;
} else {
// err != TXN_OK, fdb read err
*code = MetaServiceCode::INVALID_ARGUMENT;
*msg = "ms read key error";
return;
}
}
LOG_INFO("check {} success request={}", hint, request->ShortDebugString());
return;
}

void MetaServiceImpl::check_kv(::google::protobuf::RpcController* controller,
const CheckKVRequest* request, CheckKVResponse* response,
::google::protobuf::Closure* done) {
RPC_PREPROCESS(check_kv);
instance_id = get_instance_id(resource_mgr_, request->cloud_unique_id());
if (instance_id.empty()) {
code = MetaServiceCode::INVALID_ARGUMENT;
msg = "empty instance_id";
return;
}
if (!request->has_op()) {
code = MetaServiceCode::INVALID_ARGUMENT;
msg = "op not given";
return;
}
if (!request->has_check_keys()) {
code = MetaServiceCode::INVALID_ARGUMENT;
msg = "empty check keys";
return;
}
RPC_RATE_LIMIT(check_kv);
switch (request->op()) {
case CheckKVRequest::CREATE_INDEX_AFTER_FE_COMMIT: {
check_create_table(
instance_id, request, response, &code, &msg, [](const CheckKVRequest* request) {
return std::make_tuple(request->check_keys().index_ids(), "index",
[](std::string instance_id, int64_t id) {
return recycle_index_key({instance_id, id});
});
});
break;
}
case CheckKVRequest::CREATE_PARTITION_AFTER_FE_COMMIT: {
check_create_table(
instance_id, request, response, &code, &msg, [](const CheckKVRequest* request) {
return std::make_tuple(request->check_keys().partition_ids(), "partition",
[](std::string instance_id, int64_t id) {
return recycle_partition_key({instance_id, id});
});
});
break;
}
default:
DCHECK(false);
};
}

} // namespace doris::cloud
2 changes: 1 addition & 1 deletion fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java
Original file line number Diff line number Diff line change
Expand Up @@ -475,7 +475,7 @@ public void processAlterTable(AlterTableStmt stmt) throws UserException {
DynamicPartitionUtil.checkAlterAllowed(
(OlapTable) db.getTableOrMetaException(tableName, TableType.OLAP));
}
Env.getCurrentEnv().addPartition(db, tableName, (AddPartitionClause) alterClause);
Env.getCurrentEnv().addPartition(db, tableName, (AddPartitionClause) alterClause, false, 0, true);
} else if (alterClause instanceof AddPartitionLikeClause) {
if (!((AddPartitionLikeClause) alterClause).getIsTempPartition()) {
DynamicPartitionUtil.checkAlterAllowed(
Expand Down
19 changes: 17 additions & 2 deletions fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java
Original file line number Diff line number Diff line change
Expand Up @@ -3232,8 +3232,23 @@ public void createTableAsSelect(CreateTableAsSelectStmt stmt) throws DdlExceptio
getInternalCatalog().createTableAsSelect(stmt);
}

public void addPartition(Database db, String tableName, AddPartitionClause addPartitionClause) throws DdlException {
getInternalCatalog().addPartition(db, tableName, addPartitionClause);
/**
* Adds a partition to a table
*
* @param db
* @param tableName
* @param addPartitionClause clause in the CreateTableStmt
* @param isCreateTable this call is for creating table
* @param generatedPartitionId the preset partition id for the partition to add
* @param writeEditLog whether to write an edit log for this addition
* @return PartitionPersistInfo to be written to editlog. It may be null if no partitions added.
* @throws DdlException
*/
public PartitionPersistInfo addPartition(Database db, String tableName, AddPartitionClause addPartitionClause,
boolean isCreateTable, long generatedPartitionId,
boolean writeEditLog) throws DdlException {
return getInternalCatalog().addPartition(db, tableName, addPartitionClause,
isCreateTable, generatedPartitionId, writeEditLog);
}

public void addPartitionLike(Database db, String tableName, AddPartitionLikeClause addPartitionLikeClause)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.doris.catalog.Env;
import org.apache.doris.catalog.HashDistributionInfo;
import org.apache.doris.catalog.MTMV;
import org.apache.doris.catalog.MetaIdGenerator;
import org.apache.doris.catalog.OlapTable;
import org.apache.doris.catalog.Partition;
import org.apache.doris.catalog.PartitionItem;
Expand All @@ -45,11 +46,14 @@
import org.apache.doris.common.FeConstants;
import org.apache.doris.common.Pair;
import org.apache.doris.common.util.AutoBucketUtils;
import org.apache.doris.common.util.DebugPointUtil;
import org.apache.doris.common.util.DynamicPartitionUtil;
import org.apache.doris.common.util.MasterDaemon;
import org.apache.doris.common.util.PropertyAnalyzer;
import org.apache.doris.common.util.RangeUtils;
import org.apache.doris.common.util.TimeUtils;
import org.apache.doris.datasource.InternalCatalog;
import org.apache.doris.persist.PartitionPersistInfo;
import org.apache.doris.thrift.TStorageMedium;

import com.google.common.base.Strings;
Expand All @@ -71,6 +75,7 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;

/**
* This class is used to periodically add or drop partition on an olapTable which specify dynamic partition properties
Expand Down Expand Up @@ -582,9 +587,60 @@ private void executeDynamicPartition(Collection<Pair<Long, Long>> dynamicPartiti
}

if (!skipAddPartition) {
for (AddPartitionClause addPartitionClause : addPartitionClauses) {
// get partitionIds and indexIds
List<Long> indexIds = new ArrayList<>(olapTable.getCopiedIndexIdToMeta().keySet());
List<Long> generatedPartitionIds = new ArrayList<>();
if (executeFirstTime && Config.isCloudMode() && !addPartitionClauses.isEmpty()) {
AddPartitionClause addPartitionClause = addPartitionClauses.get(0);
DistributionDesc distributionDesc = addPartitionClause.getDistributionDesc();
try {
Env.getCurrentEnv().addPartition(db, tableName, addPartitionClause);
DistributionInfo distributionInfo = distributionDesc
.toDistributionInfo(olapTable.getBaseSchema());
if (distributionDesc == null) {
distributionInfo = olapTable.getDefaultDistributionInfo()
.toDistributionDesc().toDistributionInfo(olapTable.getBaseSchema());
}
long allPartitionBufferSize = 0;
for (int i = 0; i < addPartitionClauses.size(); i++) {
long bufferSize = InternalCatalog.checkAndGetBufferSize(indexIds.size(),
distributionInfo.getBucketNum(),
addPartitionClause.getSingeRangePartitionDesc()
.getReplicaAlloc().getTotalReplicaNum(),
db, tableName);
allPartitionBufferSize += bufferSize;
}
MetaIdGenerator.IdGeneratorBuffer idGeneratorBuffer = Env.getCurrentEnv()
.getIdGeneratorBuffer(allPartitionBufferSize);
addPartitionClauses.forEach(p -> generatedPartitionIds.add(idGeneratorBuffer.getNextId()));
// executeFirstTime true
Env.getCurrentInternalCatalog().beforeCreatePartitions(db.getId(), olapTable.getId(),
generatedPartitionIds, indexIds, true);
} catch (Exception e) {
LOG.warn("cloud in prepare step, dbName {}, tableName {}, tableId {} indexId {} exception {}",
db.getFullName(), tableName, olapTable.getId(), indexIds, e.getMessage());
recordCreatePartitionFailedMsg(db.getFullName(), tableName, e.getMessage(), olapTable.getId());
throw new DdlException("cloud in prepare step err");
}
}

List<PartitionPersistInfo> partsInfo = new ArrayList<>();
for (int i = 0; i < addPartitionClauses.size(); i++) {
try {
boolean needWriteEditLog = true;
// ATTN: !executeFirstTime, needWriteEditLog
// here executeFirstTime is create table, so in cloud edit log will postpone
if (Config.isCloudMode()) {
needWriteEditLog = !executeFirstTime;
}
PartitionPersistInfo info =
Env.getCurrentEnv().addPartition(db, tableName, addPartitionClauses.get(i),
executeFirstTime,
executeFirstTime && Config.isCloudMode() ? generatedPartitionIds.get(i) : 0,
needWriteEditLog);
if (info == null) {
throw new Exception("null persisted partition returned");
}
partsInfo.add(info);
clearCreatePartitionFailedMsg(olapTable.getId());
} catch (Exception e) {
recordCreatePartitionFailedMsg(db.getFullName(), tableName, e.getMessage(), olapTable.getId());
Expand All @@ -594,6 +650,55 @@ private void executeDynamicPartition(Collection<Pair<Long, Long>> dynamicPartiti
}
}
}
List<Long> succeedPartitionIds = partsInfo.stream().map(partitionPersistInfo
-> partitionPersistInfo.getPartition().getId()).collect(Collectors.toList());
if (executeFirstTime && Config.isCloudMode() && !addPartitionClauses.isEmpty()) {
try {
// ATTN: failedPids = generatedPartitionIds - succeedPartitionIds,
// means some partitions failed when addPartition, failedPids will be recycled by recycler
if (DebugPointUtil.isEnable("FE.DynamicPartitionScheduler.before.commitCloudPartition")) {
LOG.info("debug point FE.DynamicPartitionScheduler.before.commitCloudPartition, throw e");
// not commit, not log edit
throw new Exception("debug point FE.DynamicPartitionScheduler.before.commitCloudPartition");
}
Env.getCurrentInternalCatalog().afterCreatePartitions(db.getId(), olapTable.getId(),
succeedPartitionIds, indexIds, true);
LOG.info("begin write edit log to add partitions in batch, "
+ "numPartitions: {}, db: {}, table: {}, tableId: {}",
partsInfo.size(), db.getFullName(), tableName, olapTable.getId());
// ATTN: here, edit log must after commit cloud partition,
// prevent commit RPC failure from causing data loss
if (DebugPointUtil.isEnable("FE.DynamicPartitionScheduler.before.logEditPartitions")) {
LOG.info("debug point FE.DynamicPartitionScheduler.before.logEditPartitions, throw e");
// committed, but not log edit
throw new Exception("debug point FE.DynamicPartitionScheduler.before.commitCloudPartition");
}
for (int i = 0; i < partsInfo.size(); i++) {
Env.getCurrentEnv().getEditLog().logAddPartition(partsInfo.get(i));
if (DebugPointUtil.isEnable("FE.DynamicPartitionScheduler.in.logEditPartitions")) {
if (i == partsInfo.size() / 2) {
LOG.info("debug point FE.DynamicPartitionScheduler.in.logEditPartitions, throw e");
// committed, but log some edit, others failed
throw new Exception("debug point FE.DynamicPartitionScheduler"
+ ".in.commitCloudPartition");
}
}
}
LOG.info("finish write edit log to add partitions in batch, "
+ "numPartitions: {}, db: {}, table: {}, tableId: {}",
partsInfo.size(), db.getFullName(), tableName, olapTable.getId());
} catch (Exception e) {
LOG.warn("cloud in commit step, dbName {}, tableName {}, tableId {} exception {}",
db.getFullName(), tableName, olapTable.getId(), e.getMessage());
recordCreatePartitionFailedMsg(db.getFullName(), tableName, e.getMessage(), olapTable.getId());
throw new DdlException("cloud in commit step err");
}
}
// cloud mode, check recycle key not remained
if (Config.isCloudMode() && executeFirstTime) {
Env.getCurrentInternalCatalog().checkCreatePartitions(db.getId(), olapTable.getId(),
succeedPartitionIds, indexIds);
}
}
}
}
Expand Down
Loading

0 comments on commit 7d4ce60

Please sign in to comment.