Skip to content

Commit

Permalink
[pick](branch-2.0) pick prs from branch-2.0 (#28176)
Browse files Browse the repository at this point in the history
* [fix](hudi-catalog) fix hudi catalog code (#27966)

backport: #27963

* [fix](Nereids): fix datetime fold-constant-be in Branch-2.0 (#27938)

fix fold-constant-be datetimev2 will lose the scale of datetime.

* [bug](function) fix compound expr coredump problem #27988 (#27989)

* [branch-2.0](cache) Fix partition cache support DATEV2  #27978 (#27990)

* [branch-2.0] delete auto partition docs (#28001)

* [fix](stats) Drop stats or update updated rows after truncate table (#27956)

1. Also clear follower's stats cache when doing drop stats.
2. Drop stats when truncate a table.

backport #27931

* [minor](stats) truncate min/max if too long #27955 (#27997)

* [agg](profile) fix incorrect profile (#28004) (#28035)

* [fix](remote-scanner-pool) missing _remote_thread_pool_max_size value #28057 (#28053)

bp #28057

* [enhance](partitionid) check partition id to avoid unexpected behavior (#28045)

* [opt](resource-tag) root and admin user can use any resource tag by default #28088 (#28080)

bp #28088

* [compile](compile option) add deflate to X86 only (#28089)

* change version to 2.0.3-rc05-var05

* [fix](Nereids): Preserve `""` in single quote strings and `''` in double quote strings. (#27959)

---------

Co-authored-by: Tiewei Fang <[email protected]>
Co-authored-by: jakevin <[email protected]>
Co-authored-by: HappenLee <[email protected]>
Co-authored-by: Xinyi Zou <[email protected]>
Co-authored-by: zclllyybb <[email protected]>
Co-authored-by: Jibing-Li <[email protected]>
Co-authored-by: AKIRA <[email protected]>
Co-authored-by: TengJianPing <[email protected]>
Co-authored-by: Mingyu Chen <[email protected]>
Co-authored-by: Yongqiang YANG <[email protected]>
Co-authored-by: shuke <[email protected]>
Co-authored-by: 谢健 <[email protected]>
  • Loading branch information
13 people authored Dec 8, 2023
1 parent 21303b9 commit 9d5f314
Show file tree
Hide file tree
Showing 45 changed files with 199 additions and 552 deletions.
8 changes: 7 additions & 1 deletion be/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -559,9 +559,15 @@ set(COMMON_THIRDPARTY
xml2
lzma
simdjson
deflate
)

if (ARCH_AMD64)
set(COMMON_THIRDPARTY
${COMMON_THIRDPARTY}
deflate
)
endif()

if ((ARCH_AMD64 OR ARCH_AARCH64) AND OS_LINUX)
add_library(hadoop_hdfs STATIC IMPORTED)
set_target_properties(hadoop_hdfs PROPERTIES IMPORTED_LOCATION ${THIRDPARTY_DIR}/lib/hadoop_hdfs/native/libhdfs.a)
Expand Down
17 changes: 12 additions & 5 deletions be/src/olap/data_dir.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -506,15 +506,22 @@ Status DataDir::load() {
_meta, rowset_meta->partition_id(), rowset_meta->txn_id(),
rowset_meta->tablet_id(), rowset_meta->tablet_schema_hash(),
rowset_meta->tablet_uid(), rowset_meta->load_id(), rowset, true);
if (!commit_txn_status && !commit_txn_status.is<PUSH_TRANSACTION_ALREADY_EXIST>()) {
LOG(WARNING) << "failed to add committed rowset: " << rowset_meta->rowset_id()
<< " to tablet: " << rowset_meta->tablet_id()
<< " for txn: " << rowset_meta->txn_id();
} else {
if (commit_txn_status || commit_txn_status.is<PUSH_TRANSACTION_ALREADY_EXIST>()) {
LOG(INFO) << "successfully to add committed rowset: " << rowset_meta->rowset_id()
<< " to tablet: " << rowset_meta->tablet_id()
<< " schema hash: " << rowset_meta->tablet_schema_hash()
<< " for txn: " << rowset_meta->txn_id();
} else if (commit_txn_status.is<ErrorCode::INTERNAL_ERROR>()) {
LOG(WARNING) << "failed to add committed rowset: " << rowset_meta->rowset_id()
<< " to tablet: " << rowset_meta->tablet_id()
<< " for txn: " << rowset_meta->txn_id()
<< " error: " << commit_txn_status;
return commit_txn_status;
} else {
LOG(WARNING) << "failed to add committed rowset: " << rowset_meta->rowset_id()
<< " to tablet: " << rowset_meta->tablet_id()
<< " for txn: " << rowset_meta->txn_id()
<< " error: " << commit_txn_status;
}
} else if (rowset_meta->rowset_state() == RowsetStatePB::VISIBLE &&
rowset_meta->tablet_uid() == tablet->tablet_uid()) {
Expand Down
6 changes: 6 additions & 0 deletions be/src/olap/rowset/rowset_meta_manager.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,12 @@ Status RowsetMetaManager::get_json_rowset_meta(OlapMeta* meta, TabletUid tablet_
}
Status RowsetMetaManager::save(OlapMeta* meta, TabletUid tablet_uid, const RowsetId& rowset_id,
const RowsetMetaPB& rowset_meta_pb, bool enable_binlog) {
if (rowset_meta_pb.partition_id() <= 0) {
LOG(WARNING) << "invalid partition id " << rowset_meta_pb.partition_id() << " tablet "
<< rowset_meta_pb.tablet_id();
return Status::InternalError("invaid partition id {} tablet {}",
rowset_meta_pb.partition_id(), rowset_meta_pb.tablet_id());
}
if (enable_binlog) {
return _save_with_binlog(meta, tablet_uid, rowset_id, rowset_meta_pb);
} else {
Expand Down
5 changes: 5 additions & 0 deletions be/src/olap/tablet.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -243,6 +243,11 @@ void WriteCooldownMetaExecutors::WriteCooldownMetaExecutors::submit(TabletShared

TabletSharedPtr Tablet::create_tablet_from_meta(TabletMetaSharedPtr tablet_meta,
DataDir* data_dir) {
if (tablet_meta->partition_id() <= 0) {
LOG(WARNING) << "invalid partition id " << tablet_meta->partition_id() << ", tablet "
<< tablet_meta->tablet_id();
return nullptr;
}
return std::make_shared<Tablet>(tablet_meta, data_dir);
}

Expand Down
6 changes: 6 additions & 0 deletions be/src/olap/tablet_meta_manager.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,12 @@ Status TabletMetaManager::save(DataDir* store, TTabletId tablet_id, TSchemaHash
std::string key = fmt::format("{}{}_{}", header_prefix, tablet_id, schema_hash);
std::string value;
tablet_meta->serialize(&value);
if (tablet_meta->partition_id() <= 0) {
LOG(WARNING) << "invalid partition id " << tablet_meta->partition_id() << " tablet "
<< tablet_meta->tablet_id();
return Status::InternalError("invaid partition id {} tablet {}",
tablet_meta->partition_id(), tablet_meta->tablet_id());
}
OlapMeta* meta = store->get_meta();
VLOG_NOTICE << "save tablet meta"
<< ", key:" << key << ", meta length:" << value.length();
Expand Down
7 changes: 4 additions & 3 deletions be/src/olap/txn_manager.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -257,9 +257,10 @@ Status TxnManager::commit_txn(OlapMeta* meta, TPartitionId partition_id,
const PUniqueId& load_id, const RowsetSharedPtr& rowset_ptr,
bool is_recovery) {
if (partition_id < 1 || transaction_id < 1 || tablet_id < 1) {
LOG(FATAL) << "invalid commit req "
<< " partition_id=" << partition_id << " transaction_id=" << transaction_id
<< " tablet_id=" << tablet_id;
LOG(WARNING) << "invalid commit req "
<< " partition_id=" << partition_id << " transaction_id=" << transaction_id
<< " tablet_id=" << tablet_id;
return Status::InternalError("invalid partition id");
}

pair<int64_t, int64_t> key(partition_id, transaction_id);
Expand Down
1 change: 0 additions & 1 deletion be/src/vec/exec/distinct_vaggregation_node.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@ DistinctAggregationNode::DistinctAggregationNode(ObjectPool* pool, const TPlanNo

Status DistinctAggregationNode::_distinct_pre_agg_with_serialized_key(
doris::vectorized::Block* in_block, doris::vectorized::Block* out_block) {
SCOPED_TIMER(_build_timer);
DCHECK(!_probe_expr_ctxs.empty());

size_t key_size = _probe_expr_ctxs.size();
Expand Down
7 changes: 5 additions & 2 deletions be/src/vec/exec/scan/scanner_scheduler.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -116,9 +116,12 @@ Status ScannerScheduler::init(ExecEnv* env) {
config::doris_scanner_thread_pool_queue_size, "local_scan");

// 3. remote scan thread pool
_remote_thread_pool_max_size = config::doris_max_remote_scanner_thread_pool_thread_num != -1
? config::doris_max_remote_scanner_thread_pool_thread_num
: std::max(512, CpuInfo::num_cores() * 10);
_remote_scan_thread_pool = std::make_unique<PriorityThreadPool>(
config::doris_remote_scanner_thread_pool_thread_num,
config::doris_remote_scanner_thread_pool_queue_size, "RemoteScanThreadPool");
_remote_thread_pool_max_size, config::doris_remote_scanner_thread_pool_queue_size,
"RemoteScanThreadPool");

// 4. limited scan thread pool
ThreadPoolBuilder("LimitedScanThreadPool")
Expand Down
13 changes: 7 additions & 6 deletions be/src/vec/exec/vaggregation_node.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -106,7 +106,6 @@ AggregationNode::AggregationNode(ObjectPool* pool, const TPlanNode& tnode,
: ExecNode(pool, tnode, descs),
_hash_table_compute_timer(nullptr),
_hash_table_input_counter(nullptr),
_build_timer(nullptr),
_expr_timer(nullptr),
_exec_timer(nullptr),
_intermediate_tuple_id(tnode.agg_node.intermediate_tuple_id),
Expand Down Expand Up @@ -332,7 +331,6 @@ Status AggregationNode::prepare_profile(RuntimeState* state) {
_serialize_key_arena_memory_usage = runtime_profile()->AddHighWaterMarkCounter(
"SerializeKeyArena", TUnit::BYTES, "MemoryUsage");

_build_timer = ADD_TIMER(runtime_profile(), "BuildTime");
_build_table_convert_timer = ADD_TIMER(runtime_profile(), "BuildConvertToPartitionedTime");
_serialize_key_timer = ADD_TIMER(runtime_profile(), "SerializeKeyTime");
_exec_timer = ADD_TIMER(runtime_profile(), "ExecTime");
Expand Down Expand Up @@ -375,7 +373,7 @@ Status AggregationNode::prepare_profile(RuntimeState* state) {

// set profile timer to evaluators
for (auto& evaluator : _aggregate_evaluators) {
evaluator->set_timer(_exec_timer, _merge_timer, _expr_timer);
evaluator->set_timer(_merge_timer, _expr_timer);
}

_offsets_of_aggregate_states.resize(_aggregate_evaluators.size());
Expand Down Expand Up @@ -486,19 +484,21 @@ Status AggregationNode::prepare_profile(RuntimeState* state) {
_is_merge ? "true" : "false", _needs_finalize ? "true" : "false",
_is_streaming_preagg ? "true" : "false",
std::to_string(_aggregate_evaluators.size()), std::to_string(_limit));
runtime_profile()->add_info_string("AggInfos:", fmt::to_string(msg));
runtime_profile()->add_info_string("AggInfos", fmt::to_string(msg));
return Status::OK();
}

Status AggregationNode::prepare(RuntimeState* state) {
SCOPED_TIMER(_runtime_profile->total_time_counter());

RETURN_IF_ERROR(ExecNode::prepare(state));
SCOPED_TIMER(_exec_timer);
RETURN_IF_ERROR(prepare_profile(state));
return Status::OK();
}

Status AggregationNode::alloc_resource(doris::RuntimeState* state) {
SCOPED_TIMER(_exec_timer);
RETURN_IF_ERROR(ExecNode::alloc_resource(state));

RETURN_IF_ERROR(VExpr::open(_probe_expr_ctxs, state));
Expand Down Expand Up @@ -547,6 +547,7 @@ Status AggregationNode::open(RuntimeState* state) {

Status AggregationNode::do_pre_agg(vectorized::Block* input_block,
vectorized::Block* output_block) {
SCOPED_TIMER(_exec_timer);
RETURN_IF_ERROR(_executor.pre_agg(input_block, output_block));

// pre stream agg need use _num_row_return to decide whether to do pre stream agg
Expand Down Expand Up @@ -585,6 +586,7 @@ Status AggregationNode::get_next(RuntimeState* state, Block* block, bool* eos) {
}

Status AggregationNode::pull(doris::RuntimeState* state, vectorized::Block* block, bool* eos) {
SCOPED_TIMER(_exec_timer);
RETURN_IF_ERROR(_executor.get_result(state, block, eos));
_make_nullable_output_key(block);
// dispose the having clause, should not be execute in prestreaming agg
Expand All @@ -595,6 +597,7 @@ Status AggregationNode::pull(doris::RuntimeState* state, vectorized::Block* bloc
}

Status AggregationNode::sink(doris::RuntimeState* state, vectorized::Block* in_block, bool eos) {
SCOPED_TIMER(_exec_timer);
if (in_block->rows() > 0) {
RETURN_IF_ERROR(_executor.execute(in_block));
RETURN_IF_ERROR(_try_spill_disk());
Expand Down Expand Up @@ -744,7 +747,6 @@ Status AggregationNode::_serialize_without_key(RuntimeState* state, Block* block

Status AggregationNode::_execute_without_key(Block* block) {
DCHECK(_agg_data->without_key != nullptr);
SCOPED_TIMER(_build_timer);
for (int i = 0; i < _aggregate_evaluators.size(); ++i) {
RETURN_IF_ERROR(_aggregate_evaluators[i]->execute_single_add(
block, _agg_data->without_key + _offsets_of_aggregate_states[i],
Expand Down Expand Up @@ -1059,7 +1061,6 @@ void AggregationNode::_find_in_hash_table(AggregateDataPtr* places, ColumnRawPtr

Status AggregationNode::_pre_agg_with_serialized_key(doris::vectorized::Block* in_block,
doris::vectorized::Block* out_block) {
SCOPED_TIMER(_build_timer);
DCHECK(!_probe_expr_ctxs.empty());

size_t key_size = _probe_expr_ctxs.size();
Expand Down
2 changes: 0 additions & 2 deletions be/src/vec/exec/vaggregation_node.h
Original file line number Diff line number Diff line change
Expand Up @@ -855,7 +855,6 @@ class AggregationNode : public ::doris::ExecNode {
std::vector<size_t> _make_nullable_keys;
RuntimeProfile::Counter* _hash_table_compute_timer;
RuntimeProfile::Counter* _hash_table_input_counter;
RuntimeProfile::Counter* _build_timer;
RuntimeProfile::Counter* _expr_timer;
RuntimeProfile::Counter* _exec_timer;

Expand Down Expand Up @@ -973,7 +972,6 @@ class AggregationNode : public ::doris::ExecNode {
private:
template <bool limit>
Status _execute_with_serialized_key_helper(Block* block) {
SCOPED_TIMER(_build_timer);
DCHECK(!_probe_expr_ctxs.empty());

size_t key_size = _probe_expr_ctxs.size();
Expand Down
2 changes: 1 addition & 1 deletion be/src/vec/exprs/vcompound_pred.h
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,7 @@ class VCompoundPred : public VectorizedFnCall {
return res_id;
};

auto create_null_map_column = [&](ColumnPtr null_map_column,
auto create_null_map_column = [&](ColumnPtr& null_map_column,
uint8* __restrict null_map_data) {
if (null_map_data == nullptr) {
null_map_column = ColumnUInt8::create(size, 0);
Expand Down
6 changes: 0 additions & 6 deletions be/src/vec/exprs/vectorized_agg_fn.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,6 @@ AggFnEvaluator::AggFnEvaluator(const TExprNode& desc)
_return_type(TypeDescriptor::from_thrift(desc.fn.ret_type)),
_intermediate_slot_desc(nullptr),
_output_slot_desc(nullptr),
_exec_timer(nullptr),
_merge_timer(nullptr),
_expr_timer(nullptr) {
bool nullable = true;
Expand Down Expand Up @@ -233,39 +232,34 @@ void AggFnEvaluator::destroy(AggregateDataPtr place) {

Status AggFnEvaluator::execute_single_add(Block* block, AggregateDataPtr place, Arena* arena) {
RETURN_IF_ERROR(_calc_argument_columns(block));
SCOPED_TIMER(_exec_timer);
_function->add_batch_single_place(block->rows(), place, _agg_columns.data(), arena);
return Status::OK();
}

Status AggFnEvaluator::execute_batch_add(Block* block, size_t offset, AggregateDataPtr* places,
Arena* arena, bool agg_many) {
RETURN_IF_ERROR(_calc_argument_columns(block));
SCOPED_TIMER(_exec_timer);
_function->add_batch(block->rows(), places, offset, _agg_columns.data(), arena, agg_many);
return Status::OK();
}

Status AggFnEvaluator::execute_batch_add_selected(Block* block, size_t offset,
AggregateDataPtr* places, Arena* arena) {
RETURN_IF_ERROR(_calc_argument_columns(block));
SCOPED_TIMER(_exec_timer);
_function->add_batch_selected(block->rows(), places, offset, _agg_columns.data(), arena);
return Status::OK();
}

Status AggFnEvaluator::streaming_agg_serialize(Block* block, BufferWritable& buf,
const size_t num_rows, Arena* arena) {
RETURN_IF_ERROR(_calc_argument_columns(block));
SCOPED_TIMER(_exec_timer);
_function->streaming_agg_serialize(_agg_columns.data(), buf, num_rows, arena);
return Status::OK();
}

Status AggFnEvaluator::streaming_agg_serialize_to_column(Block* block, MutableColumnPtr& dst,
const size_t num_rows, Arena* arena) {
RETURN_IF_ERROR(_calc_argument_columns(block));
SCOPED_TIMER(_exec_timer);
_function->streaming_agg_serialize_to_column(_agg_columns.data(), dst, num_rows, arena);
return Status::OK();
}
Expand Down
5 changes: 1 addition & 4 deletions be/src/vec/exprs/vectorized_agg_fn.h
Original file line number Diff line number Diff line change
Expand Up @@ -56,9 +56,7 @@ class AggFnEvaluator {
const SlotDescriptor* intermediate_slot_desc,
const SlotDescriptor* output_slot_desc);

void set_timer(RuntimeProfile::Counter* exec_timer, RuntimeProfile::Counter* merge_timer,
RuntimeProfile::Counter* expr_timer) {
_exec_timer = exec_timer;
void set_timer(RuntimeProfile::Counter* merge_timer, RuntimeProfile::Counter* expr_timer) {
_merge_timer = merge_timer;
_expr_timer = expr_timer;
}
Expand Down Expand Up @@ -120,7 +118,6 @@ class AggFnEvaluator {
const SlotDescriptor* _intermediate_slot_desc;
const SlotDescriptor* _output_slot_desc;

RuntimeProfile::Counter* _exec_timer;
RuntimeProfile::Counter* _merge_timer;
RuntimeProfile::Counter* _expr_timer;

Expand Down
2 changes: 2 additions & 0 deletions be/test/olap/delete_handler_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -100,6 +100,7 @@ static void tear_down() {
static void set_default_create_tablet_request(TCreateTabletReq* request) {
request->tablet_id = 10003;
request->__set_version(1);
request->partition_id = 10004;
request->tablet_schema.schema_hash = 270068375;
request->tablet_schema.short_key_column_count = 2;
request->tablet_schema.keys_type = TKeysType::AGG_KEYS;
Expand Down Expand Up @@ -185,6 +186,7 @@ static void set_default_create_tablet_request(TCreateTabletReq* request) {

static void set_create_duplicate_tablet_request(TCreateTabletReq* request) {
request->tablet_id = 10009;
request->partition_id = 10010;
request->__set_version(1);
request->tablet_schema.schema_hash = 270068376;
request->tablet_schema.short_key_column_count = 2;
Expand Down
2 changes: 2 additions & 0 deletions be/test/olap/delta_writer_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,7 @@ static void tear_down() {
static void create_tablet_request(int64_t tablet_id, int32_t schema_hash,
TCreateTabletReq* request) {
request->tablet_id = tablet_id;
request->partition_id = 1000;
request->__set_version(1);
request->tablet_schema.schema_hash = schema_hash;
request->tablet_schema.short_key_column_count = 6;
Expand Down Expand Up @@ -264,6 +265,7 @@ static void create_tablet_request_with_sequence_col(int64_t tablet_id, int32_t s
TCreateTabletReq* request,
bool enable_mow = false) {
request->tablet_id = tablet_id;
request->partition_id = 10000;
request->__set_version(1);
request->tablet_schema.schema_hash = schema_hash;
request->tablet_schema.short_key_column_count = 2;
Expand Down
1 change: 1 addition & 0 deletions be/test/olap/engine_storage_migration_task_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -100,6 +100,7 @@ static void tear_down() {
static void create_tablet_request_with_sequence_col(int64_t tablet_id, int32_t schema_hash,
TCreateTabletReq* request) {
request->tablet_id = tablet_id;
request->partition_id = 20001;
request->__set_version(1);
request->tablet_schema.schema_hash = schema_hash;
request->tablet_schema.short_key_column_count = 2;
Expand Down
1 change: 1 addition & 0 deletions be/test/olap/remote_rowset_gc_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -117,6 +117,7 @@ class RemoteRowsetGcTest : public testing::Test {
static void create_tablet_request_with_sequence_col(int64_t tablet_id, int32_t schema_hash,
TCreateTabletReq* request) {
request->tablet_id = tablet_id;
request->partition_id = 1000;
request->__set_version(1);
request->tablet_schema.schema_hash = schema_hash;
request->tablet_schema.short_key_column_count = 2;
Expand Down
1 change: 1 addition & 0 deletions be/test/olap/tablet_cooldown_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -276,6 +276,7 @@ class TabletCooldownTest : public testing::Test {
static void create_tablet_request_with_sequence_col(int64_t tablet_id, int32_t schema_hash,
TCreateTabletReq* request) {
request->tablet_id = tablet_id;
request->partition_id = 1000;
request->__set_version(1);
request->tablet_schema.schema_hash = schema_hash;
request->tablet_schema.short_key_column_count = 2;
Expand Down
3 changes: 3 additions & 0 deletions be/test/olap/tablet_mgr_test.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -107,6 +107,7 @@ TEST_F(TabletMgrTest, CreateTablet) {
create_tablet_req.__set_tablet_schema(tablet_schema);
create_tablet_req.__set_tablet_id(111);
create_tablet_req.__set_version(2);
create_tablet_req.__set_partition_id(1000);
std::vector<DataDir*> data_dirs;
data_dirs.push_back(_data_dir);
RuntimeProfile profile("CreateTablet");
Expand Down Expand Up @@ -167,6 +168,7 @@ TEST_F(TabletMgrTest, CreateTabletWithSequence) {
TCreateTabletReq create_tablet_req;
create_tablet_req.__set_tablet_schema(tablet_schema);
create_tablet_req.__set_tablet_id(111);
create_tablet_req.__set_partition_id(1000);
create_tablet_req.__set_version(2);
std::vector<DataDir*> data_dirs;
data_dirs.push_back(_data_dir);
Expand Down Expand Up @@ -210,6 +212,7 @@ TEST_F(TabletMgrTest, DropTablet) {
TCreateTabletReq create_tablet_req;
create_tablet_req.__set_tablet_schema(tablet_schema);
create_tablet_req.__set_tablet_id(111);
create_tablet_req.__set_partition_id(1000);
create_tablet_req.__set_version(2);
std::vector<DataDir*> data_dirs;
data_dirs.push_back(_data_dir);
Expand Down
Loading

0 comments on commit 9d5f314

Please sign in to comment.