Remove the restriction of max_table_load_session_count in direct-load data
This commit is contained in:
parent
907db4b32b
commit
0ff05b2832
@ -104,6 +104,7 @@ int ObTableLoadBeginP::process()
|
||||
param.tenant_id_ = credential_.tenant_id_;
|
||||
param.table_id_ = table_id;
|
||||
param.batch_size_ = arg_.config_.batch_size_;
|
||||
param.parallel_ = arg_.config_.session_count_;
|
||||
param.session_count_ = MIN(arg_.config_.session_count_, (int32_t)tenant->unit_max_cpu());
|
||||
param.max_error_row_count_ = arg_.config_.max_error_row_count_;
|
||||
param.column_count_ = column_names.count();
|
||||
@ -200,7 +201,7 @@ int ObTableLoadBeginP::create_table_ctx(const ObTableLoadParam ¶m,
|
||||
uint64_t data_version = 0;
|
||||
start_arg.tenant_id_ = param.tenant_id_;
|
||||
start_arg.table_id_ = param.table_id_;
|
||||
start_arg.parallelism_ = param.session_count_;
|
||||
start_arg.parallelism_ = param.parallel_;
|
||||
if (OB_FAIL(GET_MIN_DATA_VERSION(param.tenant_id_, data_version))) {
|
||||
LOG_WARN("fail to get tenant data version", KR(ret));
|
||||
} else if (OB_FAIL(ObTableLoadRedefTable::start(start_arg, start_res, session_info))) {
|
||||
@ -271,6 +272,7 @@ int ObTableLoadPreBeginPeerP::process()
|
||||
param.tenant_id_ = credential_.tenant_id_;
|
||||
param.table_id_ = arg_.table_id_;
|
||||
param.batch_size_ = arg_.config_.batch_size_;
|
||||
param.parallel_ = arg_.config_.session_count_;
|
||||
param.session_count_ = arg_.config_.session_count_;
|
||||
param.max_error_row_count_ = arg_.config_.max_error_row_count_;
|
||||
param.column_count_ = arg_.column_count_;
|
||||
|
@ -103,7 +103,7 @@ int ObTableLoadInstance::create_table_ctx(ObTableLoadParam ¶m,
|
||||
uint64_t data_version = 0;
|
||||
start_arg.tenant_id_ = param.tenant_id_;
|
||||
start_arg.table_id_ = param.table_id_;
|
||||
start_arg.parallelism_ = param.session_count_;
|
||||
start_arg.parallelism_ = param.parallel_;
|
||||
start_arg.is_load_data_ = !param.px_mode_;
|
||||
if (OB_FAIL(GET_MIN_DATA_VERSION(param.tenant_id_, data_version))) {
|
||||
LOG_WARN("fail to get tenant data version", KR(ret));
|
||||
|
@ -844,9 +844,6 @@ int ObTableLoadStore::px_start_trans(const ObTableLoadTransId &trans_id)
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObTableLoadStore not init", KR(ret), KP(this));
|
||||
} else if (OB_UNLIKELY(trans_id.segment_id_.id_ > param_.session_count_)) {
|
||||
ret = OB_NOT_SUPPORTED;
|
||||
LOG_WARN("not support trans id", KR(ret), K(trans_id), K(param_.session_count_));
|
||||
} else {
|
||||
LOG_INFO("store px start trans", K(trans_id));
|
||||
ObTableLoadStoreTrans *trans = nullptr;
|
||||
@ -914,7 +911,8 @@ int ObTableLoadStore::px_write(const ObTableLoadTransId &trans_id,
|
||||
} else {
|
||||
if (OB_SUCC(trans->check_trans_status(ObTableLoadTransStatusType::RUNNING)) ||
|
||||
OB_SUCC(trans->check_trans_status(ObTableLoadTransStatusType::FROZEN))) {
|
||||
if (OB_FAIL(store_writer->write(trans_id.segment_id_.id_, tablet_id, row_array))) {
|
||||
int32_t session_id = 1; // in px mode, each trans contains only 1 session
|
||||
if (OB_FAIL(store_writer->write(session_id, tablet_id, row_array))) {
|
||||
LOG_WARN("fail to write store", KR(ret));
|
||||
} else {
|
||||
LOG_DEBUG("succeed to write store", K(trans_id), K(tablet_id));
|
||||
@ -936,6 +934,7 @@ int ObTableLoadStore::px_write(const ObTableLoadTransId &trans_id,
|
||||
int ObTableLoadStore::px_flush(ObTableLoadStoreTrans *trans)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int32_t session_id = 1;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObTableLoadStore not init", KR(ret), KP(this));
|
||||
@ -947,7 +946,7 @@ int ObTableLoadStore::px_flush(ObTableLoadStoreTrans *trans)
|
||||
// after get store writer, avoid early commit
|
||||
else if (OB_FAIL(trans->set_trans_status_frozen())) {
|
||||
LOG_WARN("fail to freeze trans", KR(ret));
|
||||
} else if (OB_FAIL(store_writer->flush(trans->get_trans_id().segment_id_.id_))) {
|
||||
} else if (OB_FAIL(store_writer->flush(session_id))) {
|
||||
LOG_WARN("fail to flush store", KR(ret));
|
||||
} else {
|
||||
LOG_DEBUG("succeed to flush store");
|
||||
@ -999,6 +998,7 @@ int ObTableLoadStore::px_abandon_trans(ObTableLoadTableCtx *ctx, const ObTableLo
|
||||
int ObTableLoadStore::px_clean_up_trans(ObTableLoadStoreTrans *trans)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int32_t session_id = 1;
|
||||
if (OB_ISNULL(trans)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid args", KR(ret), KP(trans));
|
||||
@ -1006,7 +1006,7 @@ int ObTableLoadStore::px_clean_up_trans(ObTableLoadStoreTrans *trans)
|
||||
ObTableLoadTransStoreWriter *store_writer = nullptr;
|
||||
if (OB_FAIL(trans->get_store_writer(store_writer))) {
|
||||
LOG_WARN("fail to get store writer", KR(ret));
|
||||
} else if (OB_FAIL(store_writer->clean_up(trans->get_trans_id().segment_id_.id_))) {
|
||||
} else if (OB_FAIL(store_writer->clean_up(session_id))) {
|
||||
LOG_WARN("fail to clean up store writer", KR(ret));
|
||||
}
|
||||
if (OB_NOT_NULL(store_writer)) {
|
||||
|
@ -45,6 +45,7 @@ ObTableLoadStoreCtx::ObTableLoadStoreCtx(ObTableLoadTableCtx *ctx)
|
||||
fast_heap_table_ctx_(nullptr),
|
||||
tmp_file_mgr_(nullptr),
|
||||
error_row_handler_(nullptr),
|
||||
next_session_id_(0),
|
||||
allocator_("TLD_StoreCtx", OB_MALLOC_NORMAL_BLOCK_SIZE, ctx->param_.tenant_id_),
|
||||
status_(ObTableLoadStatusType::NONE),
|
||||
error_code_(OB_SUCCESS),
|
||||
|
@ -143,6 +143,7 @@ public:
|
||||
storage::ObDirectLoadTmpFileManager *tmp_file_mgr_;
|
||||
ObTableLoadErrorRowHandler *error_row_handler_;
|
||||
share::schema::ObSequenceSchema sequence_schema_;
|
||||
uint64_t next_session_id_ CACHE_ALIGNED;
|
||||
struct SessionContext
|
||||
{
|
||||
SessionContext() : extra_buf_(nullptr), extra_buf_size_(0) {}
|
||||
|
@ -19,8 +19,6 @@ class ObObj;
|
||||
namespace observer
|
||||
{
|
||||
|
||||
static const int32_t MAX_TABLE_LOAD_SESSION_COUNT = 128;
|
||||
|
||||
enum class ObTableLoadDataType : int32_t
|
||||
{
|
||||
OBJ_ARRAY = 0, //obobj[]
|
||||
@ -103,6 +101,7 @@ struct ObTableLoadParam
|
||||
ObTableLoadParam()
|
||||
: tenant_id_(common::OB_INVALID_ID),
|
||||
table_id_(common::OB_INVALID_ID),
|
||||
parallel_(0),
|
||||
session_count_(0),
|
||||
batch_size_(0),
|
||||
max_error_row_count_(0),
|
||||
@ -131,17 +130,19 @@ struct ObTableLoadParam
|
||||
{
|
||||
return common::OB_INVALID_ID != tenant_id_ &&
|
||||
common::OB_INVALID_ID != table_id_ &&
|
||||
session_count_ > 0 && session_count_ <= MAX_TABLE_LOAD_SESSION_COUNT &&
|
||||
parallel_ > 0 &&
|
||||
session_count_ > 0 &&
|
||||
batch_size_ > 0 &&
|
||||
column_count_ > 0;
|
||||
}
|
||||
|
||||
TO_STRING_KV(K_(tenant_id), K_(table_id), K_(session_count), K_(batch_size),
|
||||
TO_STRING_KV(K_(tenant_id), K_(table_id), K_(parallel), K_(session_count), K_(batch_size),
|
||||
K_(max_error_row_count), K_(sql_mode), K_(column_count), K_(need_sort), K_(px_mode),
|
||||
K_(online_opt_stat_gather), K_(data_type), K_(dup_action));
|
||||
public:
|
||||
uint64_t tenant_id_;
|
||||
uint64_t table_id_;
|
||||
int64_t parallel_;
|
||||
int32_t session_count_;
|
||||
int32_t batch_size_;
|
||||
uint64_t max_error_row_count_;
|
||||
|
@ -38,14 +38,19 @@ using namespace table;
|
||||
int ObTableLoadTransStore::init()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const int32_t session_count = trans_ctx_->ctx_->param_.session_count_;
|
||||
const int32_t session_count = trans_ctx_->ctx_->param_.px_mode_?
|
||||
1 : trans_ctx_->ctx_->param_.session_count_;
|
||||
SessionStore *session_store = nullptr;
|
||||
for (int32_t i = 0; OB_SUCC(ret) && i < session_count; ++i) {
|
||||
if (OB_ISNULL(session_store = OB_NEWx(SessionStore, (&trans_ctx_->allocator_)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("fail to new SessionStore", KR(ret));
|
||||
} else {
|
||||
session_store->session_id_ = i + 1;
|
||||
if (trans_ctx_->ctx_->param_.px_mode_) {
|
||||
session_store->session_id_ = (ATOMIC_FAA(&(trans_ctx_->ctx_->store_ctx_->next_session_id_), 1) % trans_ctx_->ctx_->param_.session_count_) + 1;
|
||||
} else {
|
||||
session_store->session_id_ = i + 1;
|
||||
}
|
||||
session_store->allocator_.set_tenant_id(trans_ctx_->ctx_->param_.tenant_id_);
|
||||
session_store->partition_table_array_.set_block_allocator(
|
||||
ModulePageAllocator(session_store->allocator_));
|
||||
@ -90,7 +95,9 @@ ObTableLoadTransStoreWriter::SessionContext::SessionContext(int32_t session_id,
|
||||
: session_id_(session_id),
|
||||
cast_allocator_("TLD_TS_Caster", OB_MALLOC_NORMAL_BLOCK_SIZE, tenant_id),
|
||||
cast_params_(cast_params),
|
||||
last_receive_sequence_no_(0)
|
||||
last_receive_sequence_no_(0),
|
||||
extra_buf_(nullptr),
|
||||
extra_buf_size_(0)
|
||||
{
|
||||
}
|
||||
|
||||
@ -114,8 +121,13 @@ ObTableLoadTransStoreWriter::ObTableLoadTransStoreWriter(ObTableLoadTransStore *
|
||||
ObTableLoadTransStoreWriter::~ObTableLoadTransStoreWriter()
|
||||
{
|
||||
if (nullptr != session_ctx_array_) {
|
||||
for (int64_t i = 0; i < param_.session_count_; ++i) {
|
||||
int32_t session_count = param_.px_mode_? 1 : param_.session_count_;
|
||||
for (int64_t i = 0; i < session_count; ++i) {
|
||||
SessionContext *session_ctx = session_ctx_array_ + i;
|
||||
if (OB_NOT_NULL(session_ctx->extra_buf_)) {
|
||||
allocator_.free(session_ctx->extra_buf_);
|
||||
session_ctx->extra_buf_ = nullptr;
|
||||
}
|
||||
session_ctx->~SessionContext();
|
||||
}
|
||||
allocator_.free(session_ctx_array_);
|
||||
@ -126,10 +138,11 @@ ObTableLoadTransStoreWriter::~ObTableLoadTransStoreWriter()
|
||||
int ObTableLoadTransStoreWriter::init()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int32_t session_count = param_.px_mode_? 1 : param_.session_count_;
|
||||
if (IS_INIT) {
|
||||
ret = OB_INIT_TWICE;
|
||||
LOG_WARN("ObTableLoadTransStoreWriter init twice", KR(ret), KP(this));
|
||||
} else if (OB_UNLIKELY(trans_store_->session_store_array_.count() != param_.session_count_)) {
|
||||
} else if (OB_UNLIKELY(trans_store_->session_store_array_.count() != session_count)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid args", KR(ret), KPC(trans_store_));
|
||||
} else {
|
||||
@ -170,15 +183,16 @@ int ObTableLoadTransStoreWriter::init_session_ctx_array()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
void *buf = nullptr;
|
||||
int32_t session_count = param_.px_mode_? 1 : param_.session_count_;
|
||||
ObDataTypeCastParams cast_params(trans_ctx_->ctx_->session_info_->get_timezone_info());
|
||||
if (OB_ISNULL(buf = allocator_.alloc(sizeof(SessionContext) * param_.session_count_))) {
|
||||
if (OB_ISNULL(buf = allocator_.alloc(sizeof(SessionContext) * session_count))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("fail to allocate memory", KR(ret));
|
||||
} else if (OB_FAIL(time_cvrt_.init(cast_params.get_nls_format(ObDateTimeType)))) {
|
||||
LOG_WARN("fail to init time converter", KR(ret));
|
||||
} else {
|
||||
session_ctx_array_ = static_cast<SessionContext *>(buf);
|
||||
for (int64_t i = 0; i < param_.session_count_; ++i) {
|
||||
for (int64_t i = 0; i < session_count; ++i) {
|
||||
new (session_ctx_array_ + i)
|
||||
SessionContext(i + 1, param_.tenant_id_, cast_params);
|
||||
}
|
||||
@ -195,20 +209,34 @@ int ObTableLoadTransStoreWriter::init_session_ctx_array()
|
||||
param.insert_table_ctx_ = trans_ctx_->ctx_->store_ctx_->insert_table_ctx_;
|
||||
param.fast_heap_table_ctx_ = trans_ctx_->ctx_->store_ctx_->fast_heap_table_ctx_;
|
||||
param.result_info_ = &(trans_ctx_->ctx_->store_ctx_->result_info_);
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < param_.session_count_; ++i) {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < session_count; ++i) {
|
||||
SessionContext *session_ctx = session_ctx_array_ + i;
|
||||
param.extra_buf_ = store_ctx_->session_ctx_array_[i].extra_buf_;
|
||||
param.extra_buf_size_ = store_ctx_->session_ctx_array_[i].extra_buf_size_;
|
||||
// init table_store_
|
||||
if (OB_FAIL(session_ctx->table_store_.init(param))) {
|
||||
LOG_WARN("fail to init table store", KR(ret));
|
||||
}
|
||||
// init datum_row_
|
||||
else if (OB_FAIL(session_ctx->datum_row_.init(table_data_desc_->column_count_))) {
|
||||
LOG_WARN("fail to init datum row", KR(ret));
|
||||
if (param_.px_mode_) {
|
||||
session_ctx->extra_buf_size_ = table_data_desc_->extra_buf_size_;
|
||||
if (OB_ISNULL(session_ctx->extra_buf_ =
|
||||
static_cast<char *>(allocator_.alloc(session_ctx->extra_buf_size_)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("fail to alloc memory", KR(ret));
|
||||
} else {
|
||||
param.extra_buf_ = session_ctx->extra_buf_;
|
||||
param.extra_buf_size_ = session_ctx->extra_buf_size_;
|
||||
}
|
||||
} else {
|
||||
session_ctx->datum_row_.row_flag_.set_flag(ObDmlFlag::DF_INSERT);
|
||||
session_ctx->datum_row_.mvcc_row_flag_.set_last_multi_version_row(true);
|
||||
param.extra_buf_ = store_ctx_->session_ctx_array_[i].extra_buf_;
|
||||
param.extra_buf_size_ = store_ctx_->session_ctx_array_[i].extra_buf_size_;
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
// init table_store_
|
||||
if (OB_FAIL(session_ctx->table_store_.init(param))) {
|
||||
LOG_WARN("fail to init table store", KR(ret));
|
||||
}
|
||||
// init datum_row_
|
||||
else if (OB_FAIL(session_ctx->datum_row_.init(table_data_desc_->column_count_))) {
|
||||
LOG_WARN("fail to init datum row", KR(ret));
|
||||
} else {
|
||||
session_ctx->datum_row_.row_flag_.set_flag(ObDmlFlag::DF_INSERT);
|
||||
session_ctx->datum_row_.mvcc_row_flag_.set_last_multi_version_row(true);
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -218,10 +246,11 @@ int ObTableLoadTransStoreWriter::advance_sequence_no(int32_t session_id, uint64_
|
||||
ObTableLoadMutexGuard &guard)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int32_t session_count = param_.px_mode_? 1 : param_.session_count_;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObTableLoadTransStoreWriter not init", KR(ret));
|
||||
} else if (OB_UNLIKELY(session_id < 1 || session_id > param_.session_count_)) {
|
||||
} else if (OB_UNLIKELY(session_id < 1 || session_id > session_count)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid args", KR(ret), K(session_id));
|
||||
} else {
|
||||
@ -280,10 +309,11 @@ int ObTableLoadTransStoreWriter::write(int32_t session_id,
|
||||
const ObTabletID &tablet_id, const ObIArray<ObNewRow> &row_array)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int32_t session_count = param_.px_mode_? 1 : param_.session_count_;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObTableLoadTransStoreWriter not init", KR(ret));
|
||||
} else if (OB_UNLIKELY(session_id < 1 || session_id > param_.session_count_) ||
|
||||
} else if (OB_UNLIKELY(session_id < 1 || session_id > session_count) ||
|
||||
row_array.empty()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid args", KR(ret), K(session_id), K(row_array.empty()));
|
||||
@ -313,10 +343,11 @@ int ObTableLoadTransStoreWriter::write(int32_t session_id,
|
||||
int ObTableLoadTransStoreWriter::flush(int32_t session_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int32_t session_count = param_.px_mode_? 1 : param_.session_count_;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObTableLoadTransStoreWriter not init", KR(ret));
|
||||
} else if (OB_UNLIKELY(session_id < 1 || session_id > param_.session_count_)) {
|
||||
} else if (OB_UNLIKELY(session_id < 1 || session_id > session_count)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid args", KR(ret), K(session_id));
|
||||
} else {
|
||||
@ -338,10 +369,11 @@ int ObTableLoadTransStoreWriter::flush(int32_t session_id)
|
||||
int ObTableLoadTransStoreWriter::clean_up(int32_t session_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int32_t session_count = param_.px_mode_? 1 : param_.session_count_;
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObTableLoadTransStoreWriter not init", KR(ret));
|
||||
} else if (OB_UNLIKELY(session_id < 1 || session_id > param_.session_count_)) {
|
||||
} else if (OB_UNLIKELY(session_id < 1 || session_id > session_count)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid args", KR(ret), K(session_id));
|
||||
} else {
|
||||
|
@ -106,6 +106,8 @@ private:
|
||||
ObDataTypeCastParams cast_params_;
|
||||
storage::ObDirectLoadTableStore table_store_;
|
||||
uint64_t last_receive_sequence_no_;
|
||||
char *extra_buf_;
|
||||
int64_t extra_buf_size_;
|
||||
};
|
||||
SessionContext *session_ctx_array_;
|
||||
int64_t ref_count_ CACHE_ALIGNED;
|
||||
|
@ -1018,7 +1018,7 @@ ObLoadDataDirectImpl::FileLoadExecutor::~FileLoadExecutor()
|
||||
task_scheduler_ = nullptr;
|
||||
}
|
||||
if (nullptr != worker_ctx_array_) {
|
||||
for (int64_t i = 0; i < execute_param_->parallel_; ++i) {
|
||||
for (int64_t i = 0; i < execute_param_->thread_count_; ++i) {
|
||||
WorkerContext *worker_ctx = worker_ctx_array_ + i;
|
||||
worker_ctx->~WorkerContext();
|
||||
}
|
||||
@ -1047,7 +1047,7 @@ int ObLoadDataDirectImpl::FileLoadExecutor::inner_init(const LoadExecuteParam &e
|
||||
// init task_scheduler_
|
||||
else if (OB_ISNULL(task_scheduler_ =
|
||||
OB_NEWx(ObTableLoadTaskThreadPoolScheduler, (execute_ctx_->allocator_),
|
||||
execute_param_->parallel_, *execute_ctx_->allocator_))) {
|
||||
execute_param_->thread_count_, *execute_ctx_->allocator_))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("fail to new ObTableLoadTaskThreadPoolScheduler", KR(ret));
|
||||
} else if (OB_FAIL(task_scheduler_->init())) {
|
||||
@ -1092,12 +1092,12 @@ int ObLoadDataDirectImpl::FileLoadExecutor::init_worker_ctx_array()
|
||||
int ret = OB_SUCCESS;
|
||||
void *buf = nullptr;
|
||||
if (OB_ISNULL(
|
||||
buf = execute_ctx_->allocator_->alloc(sizeof(WorkerContext) * execute_param_->parallel_))) {
|
||||
buf = execute_ctx_->allocator_->alloc(sizeof(WorkerContext) * execute_param_->thread_count_))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("fail to allocate memory", KR(ret));
|
||||
} else {
|
||||
worker_ctx_array_ = new (buf) WorkerContext[execute_param_->parallel_];
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < execute_param_->parallel_; ++i) {
|
||||
worker_ctx_array_ = new (buf) WorkerContext[execute_param_->thread_count_];
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < execute_param_->thread_count_; ++i) {
|
||||
WorkerContext *worker_ctx = worker_ctx_array_ + i;
|
||||
if (OB_FAIL(worker_ctx->data_parser_.init(execute_param_->data_access_param_,
|
||||
*execute_ctx_->logger_))) {
|
||||
@ -1292,7 +1292,7 @@ int ObLoadDataDirectImpl::FileLoadExecutor::process_task_handle(int64_t worker_i
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObLoadDataDirectImpl::FileLoadExecutor not init", KR(ret), KP(this));
|
||||
} else if (OB_UNLIKELY(worker_idx < 0 || worker_idx >= execute_param_->parallel_ ||
|
||||
} else if (OB_UNLIKELY(worker_idx < 0 || worker_idx >= execute_param_->thread_count_ ||
|
||||
nullptr == handle)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid args", KR(ret), KPC(execute_param_), K(worker_idx), KP(handle));
|
||||
@ -1527,7 +1527,7 @@ int ObLoadDataDirectImpl::LargeFileLoadExecutor::fill_task(TaskHandle *handle,
|
||||
|
||||
int32_t ObLoadDataDirectImpl::LargeFileLoadExecutor::get_session_id()
|
||||
{
|
||||
if (next_session_id_ > execute_param_->parallel_) {
|
||||
if (next_session_id_ > execute_param_->thread_count_) {
|
||||
next_session_id_ = 1;
|
||||
}
|
||||
return next_session_id_++;
|
||||
@ -1827,7 +1827,7 @@ int ObLoadDataDirectImpl::execute(ObExecContext &ctx, ObLoadDataStmt &load_stmt)
|
||||
DataDesc data_desc;
|
||||
data_desc.filename_ = load_args.file_name_;
|
||||
if (OB_FAIL(SimpleDataSplitUtils::split(execute_param_.data_access_param_, data_desc,
|
||||
execute_param_.parallel_, data_desc_iter))) {
|
||||
execute_param_.thread_count_, data_desc_iter))) {
|
||||
LOG_WARN("fail to split data", KR(ret));
|
||||
}
|
||||
} else {
|
||||
@ -1914,9 +1914,10 @@ int ObLoadDataDirectImpl::init_execute_param()
|
||||
hint_parallel = load_args.file_iter_.count() > 1
|
||||
? MIN(hint_parallel, load_args.file_iter_.count())
|
||||
: hint_parallel;
|
||||
execute_param_.parallel_ = MIN(hint_parallel, (int64_t)tenant->unit_max_cpu());
|
||||
execute_param_.parallel_ = hint_parallel;
|
||||
execute_param_.thread_count_ = MIN(hint_parallel, (int64_t)tenant->unit_max_cpu());
|
||||
execute_param_.data_mem_usage_limit_ =
|
||||
MIN(execute_param_.parallel_ * 2, MAX_DATA_MEM_USAGE_LIMIT);
|
||||
MIN(execute_param_.thread_count_ * 2, MAX_DATA_MEM_USAGE_LIMIT);
|
||||
}
|
||||
}
|
||||
// batch_row_count_
|
||||
@ -2077,7 +2078,8 @@ int ObLoadDataDirectImpl::init_execute_context()
|
||||
ObTableLoadParam load_param;
|
||||
load_param.tenant_id_ = execute_param_.tenant_id_;
|
||||
load_param.table_id_ = execute_param_.table_id_;
|
||||
load_param.session_count_ = execute_param_.parallel_;
|
||||
load_param.parallel_ = execute_param_.parallel_;
|
||||
load_param.session_count_ = execute_param_.thread_count_;
|
||||
load_param.batch_size_ = execute_param_.batch_row_count_;
|
||||
load_param.max_error_row_count_ = execute_param_.max_error_rows_;
|
||||
load_param.column_count_ = execute_param_.store_column_idxs_.count();
|
||||
@ -2129,7 +2131,7 @@ int ObLoadDataDirectImpl::init_logger()
|
||||
session->get_tenant_name().length(), session->get_tenant_name().ptr(),
|
||||
load_args.file_name_.length(), load_args.file_name_.ptr(),
|
||||
load_args.combined_name_.length(), load_args.combined_name_.ptr(),
|
||||
execute_param_.parallel_, execute_param_.batch_row_count_,
|
||||
execute_param_.thread_count_, execute_param_.batch_row_count_,
|
||||
ObCurTraceId::get_trace_id_str()));
|
||||
OZ(databuff_printf(buf, buf_len, pos, "Start time:\t"));
|
||||
OZ(ObTimeConverter::datetime_to_str(current_time, TZ_INFO(session), ObString(),
|
||||
|
@ -67,7 +67,7 @@ private:
|
||||
public:
|
||||
LoadExecuteParam();
|
||||
bool is_valid() const;
|
||||
TO_STRING_KV(K_(tenant_id), K_(database_id), K_(table_id), K_(combined_name), K_(parallel),
|
||||
TO_STRING_KV(K_(tenant_id), K_(database_id), K_(table_id), K_(combined_name), K_(parallel), K_(thread_count),
|
||||
K_(batch_row_count), K_(data_mem_usage_limit), K_(need_sort), K_(online_opt_stat_gather),
|
||||
K_(max_error_rows), K_(ignore_row_num), K_(data_access_param), K_(store_column_idxs));
|
||||
public:
|
||||
@ -78,7 +78,8 @@ private:
|
||||
common::ObString database_name_;
|
||||
common::ObString table_name_;
|
||||
common::ObString combined_name_; // database name + table name
|
||||
int64_t parallel_; // number of concurrent threads
|
||||
int64_t parallel_;
|
||||
int64_t thread_count_; // number of concurrent threads
|
||||
int64_t batch_row_count_;
|
||||
int64_t data_mem_usage_limit_; // limit = data_mem_usage_limit * MAX_BUFFER_SIZE
|
||||
bool need_sort_;
|
||||
|
@ -10,6 +10,7 @@
|
||||
#include "observer/table_load/ob_table_load_struct.h"
|
||||
#include "observer/table_load/ob_table_load_instance.h"
|
||||
#include "share/schema/ob_schema_getter_guard.h"
|
||||
#include "observer/omt/ob_tenant.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
@ -48,7 +49,10 @@ int ObTableDirectInsertCtx::init(ObExecContext *exec_ctx,
|
||||
load_exec_ctx_->allocator_ = &(exec_ctx->get_allocator());
|
||||
uint64_t sql_mode = 0;
|
||||
ObSEArray<int64_t, 16> store_column_idxs;
|
||||
if (OB_FAIL(init_store_column_idxs(MTL_ID(), table_id, store_column_idxs))) {
|
||||
omt::ObTenant *tenant = nullptr;
|
||||
if (OB_FAIL(GCTX.omt_->get_tenant(MTL_ID(), tenant))) {
|
||||
LOG_WARN("fail to get tenant handle", KR(ret), K(MTL_ID()));
|
||||
} else if (OB_FAIL(init_store_column_idxs(MTL_ID(), table_id, store_column_idxs))) {
|
||||
LOG_WARN("failed to init store column idxs", KR(ret));
|
||||
} else if (OB_FAIL(exec_ctx->get_my_session()->get_sys_variable(SYS_VAR_SQL_MODE, sql_mode))) {
|
||||
LOG_WARN("fail to get sys variable", KR(ret));
|
||||
@ -58,7 +62,8 @@ int ObTableDirectInsertCtx::init(ObExecContext *exec_ctx,
|
||||
param.tenant_id_ = MTL_ID();
|
||||
param.table_id_ = table_id;
|
||||
param.batch_size_ = 100;
|
||||
param.session_count_ = parallel;
|
||||
param.parallel_ = parallel;
|
||||
param.session_count_ = MIN(parallel, (int64_t)tenant->unit_max_cpu());
|
||||
param.px_mode_ = true;
|
||||
param.online_opt_stat_gather_ = false;
|
||||
param.need_sort_ = true;
|
||||
|
Loading…
x
Reference in New Issue
Block a user