[FEAT MERGE]4_1_sql_feature
Co-authored-by: leslieyuchen <leslieyuchen@gmail.com> Co-authored-by: Charles0429 <xiezhenjiang@gmail.com> Co-authored-by: raywill <hustos@gmail.com>
This commit is contained in:
@ -117,6 +117,8 @@ OB_DEF_SERIALIZE(ObTableScanCtDef)
|
||||
if (OB_SUCC(ret) && has_dppr_tbl) {
|
||||
OB_UNIS_ENCODE(*das_dppr_tbl_);
|
||||
}
|
||||
OB_UNIS_ENCODE(calc_part_id_expr_);
|
||||
OB_UNIS_ENCODE(global_index_rowkey_exprs_);
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -141,6 +143,8 @@ OB_DEF_SERIALIZE_SIZE(ObTableScanCtDef)
|
||||
if (has_dppr_tbl) {
|
||||
OB_UNIS_ADD_LEN(*das_dppr_tbl_);
|
||||
}
|
||||
OB_UNIS_ADD_LEN(calc_part_id_expr_);
|
||||
OB_UNIS_ADD_LEN(global_index_rowkey_exprs_);
|
||||
return len;
|
||||
}
|
||||
|
||||
@ -182,6 +186,8 @@ OB_DEF_DESERIALIZE(ObTableScanCtDef)
|
||||
OZ(allocate_dppr_table_loc());
|
||||
OB_UNIS_DECODE(*das_dppr_tbl_);
|
||||
}
|
||||
OB_UNIS_DECODE(calc_part_id_expr_);
|
||||
OB_UNIS_DECODE(global_index_rowkey_exprs_);
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -592,6 +598,7 @@ ObTableScanOp::ObTableScanOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOp
|
||||
range_buffer_idx_(0),
|
||||
group_size_(0),
|
||||
max_group_size_(0),
|
||||
global_index_lookup_op_(NULL),
|
||||
spat_index_()
|
||||
{
|
||||
}
|
||||
@ -1019,7 +1026,7 @@ int ObTableScanOp::prepare_single_scan_range(int64_t group_idx)
|
||||
*table_rescan_allocator_ : ctx_.get_allocator());
|
||||
bool is_same_type = true; // use for extract equal pre_query_range
|
||||
if (OB_FAIL(single_equal_scan_check_type(plan_ctx->get_param_store(), is_same_type))) {
|
||||
LOG_WARN("failed to check type about single equal scan");
|
||||
LOG_WARN("failed to check type about single equal scan", K(ret));
|
||||
} else if (is_same_type && MY_CTDEF.pre_query_range_.get_is_equal_and()) {
|
||||
int64_t column_count = MY_CTDEF.pre_query_range_.get_column_count();
|
||||
size_t range_size = sizeof(ObNewRange) + sizeof(ObObj) * column_count * 2;
|
||||
@ -1097,7 +1104,7 @@ int ObTableScanOp::prepare_single_scan_range(int64_t group_idx)
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && MY_SPEC.is_vt_mapping_) {
|
||||
OZ(OB_FAIL(vt_result_converter_->convert_key_ranges(MY_INPUT.key_ranges_)));
|
||||
OZ(vt_result_converter_->convert_key_ranges(MY_INPUT.key_ranges_));
|
||||
}
|
||||
LOG_TRACE("prepare single scan range", K(ret), K(key_ranges), K(MY_INPUT.key_ranges_),
|
||||
K(MY_INPUT.ss_key_ranges_));
|
||||
@ -1114,14 +1121,12 @@ int ObTableScanOp::single_equal_scan_check_type(const ParamStore ¶m_store, b
|
||||
int64_t param_idx = equal_offs.at(i).param_idx_;
|
||||
if (equal_offs.at(i).only_pos_) {
|
||||
// do nothing
|
||||
} else if (param_idx < 0 || param_idx >= param_store.count()) {
|
||||
} else if (OB_UNLIKELY(param_idx < 0 || param_idx >= param_store.count())) {
|
||||
ret = OB_ERROR_OUT_OF_RANGE;
|
||||
LOG_WARN("out of param store");
|
||||
} else {
|
||||
if (equal_offs.at(i).pos_type_ != param_store.at(param_idx).get_type()
|
||||
&& !param_store.at(param_idx).is_null()) {
|
||||
is_same_type = false;
|
||||
}
|
||||
LOG_WARN("out of param store", K(ret), K(param_idx), K(param_store.count()));
|
||||
} else if (equal_offs.at(i).pos_type_ != param_store.at(param_idx).get_type()
|
||||
&& !param_store.at(param_idx).is_null()) {
|
||||
is_same_type = false;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -1249,6 +1254,23 @@ int ObTableScanOp::inner_open()
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && MY_SPEC.is_global_index_back()) {
|
||||
if (OB_NOT_NULL(global_index_lookup_op_)) {
|
||||
global_index_lookup_op_->destroy();
|
||||
global_index_lookup_op_->~ObGlobalIndexLookupOpImpl();
|
||||
global_index_lookup_op_ = nullptr;
|
||||
}
|
||||
void *lookup_buf = ctx_.get_allocator().alloc(sizeof(ObGlobalIndexLookupOpImpl));
|
||||
if (nullptr == lookup_buf) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("allocate memory failed", K(ret), K(lookup_buf));
|
||||
} else {
|
||||
global_index_lookup_op_ = new (lookup_buf) ObGlobalIndexLookupOpImpl(this);
|
||||
if (OB_FAIL(global_index_lookup_op_->open())) {
|
||||
LOG_WARN("failed to open global index lookup op", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -1264,6 +1286,14 @@ int ObTableScanOp::inner_close()
|
||||
LOG_WARN("close all das task failed", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && MY_SPEC.is_global_index_back()) {
|
||||
if (OB_ISNULL(global_index_lookup_op_)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid arguments",K(ret));
|
||||
} else if (OB_FAIL(global_index_lookup_op_->close())) {
|
||||
LOG_WARN("failed to get next batch",K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
iter_end_ = false;
|
||||
need_init_before_get_row_ = true;
|
||||
@ -1312,6 +1342,11 @@ void ObTableScanOp::destroy()
|
||||
vt_result_converter_->~ObVirtualTableResultConverter();
|
||||
vt_result_converter_ = nullptr;
|
||||
}
|
||||
if (OB_NOT_NULL(global_index_lookup_op_)) {
|
||||
global_index_lookup_op_->destroy();
|
||||
global_index_lookup_op_->~ObGlobalIndexLookupOpImpl();
|
||||
global_index_lookup_op_ = nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
int ObTableScanOp::fill_storage_feedback_info()
|
||||
@ -1380,6 +1415,29 @@ int ObTableScanOp::fill_storage_feedback_info()
|
||||
}
|
||||
|
||||
int ObTableScanOp::inner_rescan()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_SUCC(ret) && MY_SPEC.is_global_index_back()) {
|
||||
if (OB_FAIL(ObOperator::inner_rescan())) {
|
||||
LOG_WARN("failed to exec inner rescan",K(ret));
|
||||
} else if (OB_ISNULL(global_index_lookup_op_)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid arguments",K(ret));
|
||||
} else {
|
||||
global_index_lookup_op_->get_brs().size_ = brs_.size_ ;
|
||||
global_index_lookup_op_->get_brs().end_ = brs_.end_;
|
||||
if (OB_FAIL(global_index_lookup_op_->rescan())) {
|
||||
LOG_WARN("failed to get next batch",K(ret));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
if (OB_FAIL(inner_rescan_for_tsc())) {
|
||||
LOG_WARN("failed to get next row",K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
int ObTableScanOp::inner_rescan_for_tsc()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
input_row_cnt_ = 0;
|
||||
@ -1410,7 +1468,6 @@ int ObTableScanOp::inner_rescan()
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTableScanOp::close_and_reopen()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -1426,9 +1483,6 @@ int ObTableScanOp::close_and_reopen()
|
||||
//stmt allocator uses DAS allocator in the reopen process
|
||||
tsc_rtdef_.scan_rtdef_.stmt_allocator_.set_alloc(&das_ref_.get_das_alloc());
|
||||
tsc_rtdef_.scan_rtdef_.scan_allocator_.set_alloc(table_rescan_allocator_);
|
||||
if (tsc_rtdef_.lookup_rtdef_ != nullptr) {
|
||||
tsc_rtdef_.lookup_rtdef_->stmt_allocator_.set_alloc(&das_ref_.get_das_alloc());
|
||||
}
|
||||
MY_INPUT.key_ranges_.reuse();
|
||||
MY_INPUT.ss_key_ranges_.reuse();
|
||||
MY_INPUT.mbr_filters_.reuse();
|
||||
@ -1769,6 +1823,25 @@ int ObTableScanOp::get_next_batch_with_das(int64_t &count, int64_t capacity)
|
||||
}
|
||||
|
||||
int ObTableScanOp::inner_get_next_row_implement()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_SUCC(ret) && MY_SPEC.is_global_index_back()) {
|
||||
if (OB_ISNULL(global_index_lookup_op_)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid arguments",K(ret));
|
||||
} else if (OB_FAIL(global_index_lookup_op_->get_next_row())) {
|
||||
LOG_WARN("failed to get next batch",K(ret));
|
||||
}
|
||||
} else {
|
||||
if (OB_FAIL(inner_get_next_row_for_tsc())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("failed to get next row",K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
int ObTableScanOp::inner_get_next_row_for_tsc()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(0 == limit_param_.limit_)) {
|
||||
@ -1810,7 +1883,9 @@ int ObTableScanOp::inner_get_next_row_implement()
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
const ExprFixedArray &storage_output = MY_CTDEF.get_das_output_exprs();
|
||||
LOG_DEBUG("storage output row", "row", ROWEXPR2STR(eval_ctx_, storage_output), K(MY_CTDEF.scan_ctdef_.ref_table_id_));
|
||||
if (!MY_SPEC.is_global_index_back()) {
|
||||
LOG_DEBUG("storage output row", "row", ROWEXPR2STR(eval_ctx_, storage_output), K(MY_CTDEF.scan_ctdef_.ref_table_id_));
|
||||
}
|
||||
if (OB_FAIL(add_ddl_column_checksum())) {
|
||||
LOG_WARN("add ddl column checksum failed", K(ret));
|
||||
}
|
||||
@ -1840,8 +1915,32 @@ int ObTableScanOp::inner_get_next_row_implement()
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTableScanOp::inner_get_next_batch(const int64_t max_row_cnt)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_SUCC(ret) && MY_SPEC.is_global_index_back()) {
|
||||
int64_t count = 0;
|
||||
if (OB_ISNULL(global_index_lookup_op_)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid arguments",K(ret));
|
||||
} else {
|
||||
global_index_lookup_op_->get_brs().size_ = brs_.size_ ;
|
||||
global_index_lookup_op_->get_brs().end_ = brs_.end_;
|
||||
if (OB_FAIL(global_index_lookup_op_->get_next_rows(count, max_row_cnt))) {
|
||||
LOG_WARN("failed to get next rows",K(ret), K(max_row_cnt));
|
||||
} else {
|
||||
brs_.size_ = global_index_lookup_op_->get_brs().size_;
|
||||
brs_.end_ = global_index_lookup_op_->get_brs().end_;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
if (OB_FAIL(inner_get_next_batch_for_tsc(max_row_cnt))) {
|
||||
LOG_WARN("failed to get next row",K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
int ObTableScanOp::inner_get_next_batch_for_tsc(const int64_t max_row_cnt)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
clear_evaluated_flag();
|
||||
@ -1896,7 +1995,9 @@ int ObTableScanOp::inner_get_next_batch(const int64_t max_row_cnt)
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
const ExprFixedArray &storage_output = MY_CTDEF.get_das_output_exprs();
|
||||
PRINT_VECTORIZED_ROWS(SQL, DEBUG, eval_ctx_, storage_output, brs_.size_, K(MY_CTDEF.scan_ctdef_.ref_table_id_));
|
||||
if (!MY_SPEC.is_global_index_back()) {
|
||||
PRINT_VECTORIZED_ROWS(SQL, DEBUG, eval_ctx_, storage_output, brs_.size_, K(MY_CTDEF.scan_ctdef_.ref_table_id_));
|
||||
}
|
||||
if (OB_FAIL(add_ddl_column_checksum_batch(brs_.size_))) {
|
||||
LOG_WARN("add ddl column checksum failed", K(ret));
|
||||
}
|
||||
@ -2348,17 +2449,16 @@ int ObTableScanOp::init_ddl_column_checksum()
|
||||
found = true;
|
||||
if (is_pad_char_to_full_length(session->get_sql_mode())) {
|
||||
need_reshape = col_param->get_meta_type().is_fixed_len_char_type();
|
||||
} else {
|
||||
need_reshape = col_param->is_virtual_gen_col()
|
||||
&& col_param->get_meta_type().is_fixed_len_char_type();
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (!found) {
|
||||
ret = OB_ENTRY_NOT_EXIST;
|
||||
LOG_WARN("column not exist", K(ret), K(MY_SPEC.ddl_output_cids_.at(i)), K(i));
|
||||
} else if (OB_FAIL(col_need_reshape_.push_back(need_reshape))) {
|
||||
// if not found, the column is virtual generated column, in this scene,
|
||||
// no need reshape, because reshape in opt layer.
|
||||
need_reshape = false;
|
||||
}
|
||||
if (OB_SUCC(ret) && OB_FAIL(col_need_reshape_.push_back(need_reshape))) {
|
||||
LOG_WARN("failed to push back col need reshape", K(ret));
|
||||
}
|
||||
}
|
||||
@ -2699,7 +2799,6 @@ int ObTableScanOp::inner_get_next_row()
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
||||
int ObTableScanOp::inner_get_next_spatial_index_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -2829,5 +2928,524 @@ int ObTableScanOp::fill_generated_cellid_mbr(const ObObj &cellid, const ObObj &m
|
||||
return ret;
|
||||
}
|
||||
|
||||
ObGlobalIndexLookupOpImpl::ObGlobalIndexLookupOpImpl(ObTableScanOp *table_scan_op)
|
||||
: ObIndexLookupOpImpl(GLOBAL_INDEX, 10000 /*default_batch_row_count*/),
|
||||
table_scan_op_(table_scan_op),
|
||||
das_ref_(table_scan_op_->get_eval_ctx(), table_scan_op_->get_exec_ctx()),
|
||||
lookup_result_(),
|
||||
lookup_memctx_()
|
||||
{
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSqlCtx *sql_ctx = NULL;
|
||||
ObSQLSessionInfo *my_session = GET_MY_SESSION(table_scan_op_->get_exec_ctx());
|
||||
if (OB_ISNULL(sql_ctx = table_scan_op_->get_exec_ctx().get_sql_ctx())
|
||||
|| OB_ISNULL(sql_ctx->schema_guard_)
|
||||
|| OB_ISNULL(get_calc_part_id_expr())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid argument", K(ret), KP(sql_ctx), KP(get_calc_part_id_expr()));
|
||||
} else {
|
||||
ObMemAttr mem_attr;
|
||||
mem_attr.tenant_id_ = my_session->get_effective_tenant_id();
|
||||
mem_attr.label_ = ObModIds::OB_SQL_TABLE_LOOKUP;
|
||||
das_ref_.set_mem_attr(mem_attr);
|
||||
das_ref_.set_expr_frame_info(&table_scan_op_->get_spec().plan_->get_expr_frame_info());
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
table_scan_op_->das_ref_.set_lookup_iter(&lookup_result_);
|
||||
}
|
||||
if (OB_SUCC(ret) && OB_ISNULL(lookup_memctx_)) {
|
||||
lib::ContextParam param;
|
||||
param.set_mem_attr(my_session->get_effective_tenant_id(),
|
||||
ObModIds::OB_SQL_TABLE_LOOKUP, ObCtxIds::DEFAULT_CTX_ID)
|
||||
.set_properties(lib::USE_TL_PAGE_OPTIONAL);
|
||||
if (OB_FAIL(CURRENT_CONTEXT->CREATE_CONTEXT(lookup_memctx_, param))) {
|
||||
LOG_WARN("create lookup mem context entity failed", K(ret));
|
||||
} else {
|
||||
table_scan_op_->tsc_rtdef_.lookup_rtdef_->scan_allocator_.set_alloc(&lookup_memctx_->get_arena_allocator());
|
||||
table_scan_op_->tsc_rtdef_.lookup_rtdef_->stmt_allocator_.set_alloc(&lookup_memctx_->get_arena_allocator());
|
||||
}
|
||||
}
|
||||
LOG_DEBUG("open table lookup", K(table_scan_op_->get_spec()));
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::close()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(das_ref_.close_all_task())) {
|
||||
LOG_WARN("close all das task failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::rescan()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (get_batch_rescan() && lookup_group_cnt_ < index_group_cnt_) {
|
||||
LOG_DEBUG("rescan in group lookup, only need to switch iterator",
|
||||
K(lookup_group_cnt_), K(index_group_cnt_));
|
||||
if (OB_FAIL(table_scan_op_->build_bnlj_params())) {
|
||||
LOG_WARN("build batch nlj params failed", K(ret));
|
||||
} else if (OB_FAIL(switch_lookup_result_iter())) {
|
||||
LOG_WARN("switch lookup result iter failed", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(das_ref_.close_all_task())) {
|
||||
LOG_WARN("failed to close all das task", K(ret));
|
||||
} else if (OB_FAIL(table_scan_op_->inner_rescan_for_tsc())) {
|
||||
LOG_WARN("rescan operator failed", K(ret));
|
||||
} else {
|
||||
reset_for_rescan();
|
||||
}
|
||||
#ifndef NDEBUG
|
||||
OX(OB_ASSERT(false == brs_.end_));
|
||||
#endif
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::get_next_row_from_index_table()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool got_row = false;
|
||||
do {
|
||||
if (OB_FAIL(table_scan_op_->inner_get_next_row_for_tsc())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get next row from child failed", K(ret));
|
||||
} else if (get_batch_rescan()) {
|
||||
//switch to next index iterator, call child's rescan
|
||||
if (OB_FAIL(table_scan_op_->inner_rescan_for_tsc())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("rescan index operator failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
inc_index_group_cnt();
|
||||
LOG_DEBUG("switch to next index batch to fetch rowkey", K(get_index_group_cnt()), K(lookup_rowkey_cnt_));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
got_row = true;
|
||||
}
|
||||
} while (OB_SUCC(ret)&& !got_row) ;
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::process_data_table_rowkey()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObObjectID partition_id = ObExprCalcPartitionId::NONE_PARTITION_ID;
|
||||
ObTabletID tablet_id;
|
||||
ObDASScanOp *das_scan_op = nullptr;
|
||||
ObDASTabletLoc *tablet_loc = nullptr;
|
||||
|
||||
ObDASScanRtDef *lookup_rtdef = table_scan_op_->tsc_rtdef_.lookup_rtdef_;
|
||||
ObDASCtx &das_ctx = DAS_CTX(table_scan_op_->get_exec_ctx());
|
||||
if (OB_FAIL(ObExprCalcPartitionBase::calc_part_and_tablet_id(get_calc_part_id_expr(), table_scan_op_->get_eval_ctx(), partition_id, tablet_id))) {
|
||||
LOG_WARN("fail to calc part id", K(ret), KPC(get_calc_part_id_expr()));
|
||||
} else if (OB_FAIL(das_ctx.extended_tablet_loc(*lookup_rtdef->table_loc_, tablet_id, tablet_loc))) {
|
||||
LOG_WARN("pkey to tablet loc failed", K(ret));
|
||||
} else if (OB_UNLIKELY(!has_das_scan_op(tablet_loc, das_scan_op))) {
|
||||
ObDASOpType op_type = get_batch_rescan() ? DAS_OP_TABLE_BATCH_SCAN : DAS_OP_TABLE_SCAN;
|
||||
ObIDASTaskOp *tmp_op = nullptr;
|
||||
if (OB_FAIL(das_ref_.create_das_task(tablet_loc, op_type, tmp_op))) {
|
||||
LOG_WARN("prepare das task failed", K(ret));
|
||||
} else {
|
||||
das_scan_op = static_cast<ObDASScanOp*>(tmp_op);
|
||||
das_scan_op->set_scan_ctdef(get_lookup_ctdef());
|
||||
das_scan_op->set_scan_rtdef(lookup_rtdef);
|
||||
das_scan_op->set_can_part_retry(table_scan_op_->get_exec_ctx().get_my_session()->is_user_session());
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
storage::ObTableScanParam &scan_param = das_scan_op->get_scan_param();
|
||||
ObNewRange lookup_range;
|
||||
if (OB_FAIL(build_data_table_range(lookup_range))) {
|
||||
LOG_WARN("build data table range failed", K(ret), KPC(tablet_loc));
|
||||
} else if (OB_FAIL(scan_param.key_ranges_.push_back(lookup_range))) {
|
||||
LOG_WARN("store lookup key range failed", K(ret), K(scan_param));
|
||||
} else {
|
||||
scan_param.is_get_ = true;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::process_data_table_rowkeys(const int64_t size, const ObBitVector *skip)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObEvalCtx::BatchInfoScopeGuard batch_info_guard(table_scan_op_->get_eval_ctx());
|
||||
batch_info_guard.set_batch_size(size);
|
||||
for (auto i = 0; OB_SUCC(ret) && i < size; i++)
|
||||
{
|
||||
if (skip->at(i)) {
|
||||
continue;
|
||||
}
|
||||
batch_info_guard.set_batch_idx(i);
|
||||
if (OB_FAIL(process_data_table_rowkey())) {
|
||||
LOG_WARN("Failed to process_data_table_rowkey", K(ret), K(i));
|
||||
} else {
|
||||
++lookup_rowkey_cnt_;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::init_group_range(int64_t cur_group_idx, int64_t group_size)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (get_batch_rescan()) {
|
||||
for (DASTaskIter task_iter = das_ref_.begin_task_iter(); !task_iter.is_end(); ++task_iter) {
|
||||
ObDASGroupScanOp *group_op = static_cast<ObDASGroupScanOp*>(*task_iter);
|
||||
group_op->init_group_range(cur_group_idx, group_size);
|
||||
LOG_DEBUG("set group info",
|
||||
"scan_range", group_op->get_scan_param().key_ranges_,
|
||||
K(*group_op));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::do_index_lookup()
|
||||
{
|
||||
int ret = das_ref_.execute_all_task();
|
||||
if (OB_SUCC(ret)) {
|
||||
lookup_result_ = das_ref_.begin_result_iter();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::get_next_row_from_data_table()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool got_row = false;
|
||||
if (OB_UNLIKELY(lookup_result_.is_end())) {
|
||||
ret = OB_ITER_END;
|
||||
LOG_DEBUG("lookup task is empty", K(ret));
|
||||
}
|
||||
ObDASScanRtDef *lookup_rtdef = table_scan_op_->tsc_rtdef_.lookup_rtdef_;
|
||||
while (OB_SUCC(ret) && !got_row) {
|
||||
lookup_rtdef->p_pd_expr_op_->clear_datum_eval_flag();
|
||||
if (OB_FAIL(lookup_result_.get_next_row())) {
|
||||
if (OB_ITER_END == ret) {
|
||||
if (OB_FAIL(lookup_result_.next_result())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("fetch next task result failed", K(ret));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
LOG_WARN("get next row from das result failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
got_row = true;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::get_next_rows_from_data_table(int64_t &count, int64_t capacity)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(count);
|
||||
int64_t batch_size = min(capacity, table_scan_op_->get_tsc_spec().max_batch_size_);
|
||||
if (OB_FAIL(get_next_data_table_rows(brs_.size_, batch_size))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
LOG_DEBUG("get next data table rows return ITER_END", K(ret));
|
||||
} else {
|
||||
LOG_WARN("look up get next row failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::process_next_index_batch_for_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(check_lookup_row_cnt())) {
|
||||
LOG_WARN("check lookup row cnt failed", K(ret));
|
||||
} else if (need_next_index_batch()) {
|
||||
if (OB_FAIL(das_ref_.close_all_task())) {
|
||||
LOG_WARN("close all das task failed", K(ret));
|
||||
} else {
|
||||
state_ = INDEX_SCAN;
|
||||
das_ref_.reuse();
|
||||
index_end_ = false;
|
||||
if (OB_SUCC(ret) && lookup_memctx_ != nullptr) {
|
||||
lookup_memctx_->reset_remain_one_page();
|
||||
}
|
||||
}
|
||||
} else {
|
||||
state_ = FINISHED;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::process_next_index_batch_for_rows(int64_t &count)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(count);
|
||||
if (OB_FAIL(check_lookup_row_cnt())) {
|
||||
LOG_WARN("check lookup row cnt failed", K(ret));
|
||||
} else if (need_next_index_batch()) { // index search does not reach end, continue index scan
|
||||
state_ = INDEX_SCAN;
|
||||
if (OB_FAIL(das_ref_.close_all_task())) {
|
||||
LOG_WARN("close all das task failed", K(ret));
|
||||
} else {
|
||||
das_ref_.reuse();
|
||||
if (OB_SUCC(ret) && lookup_memctx_ != nullptr) {
|
||||
lookup_memctx_->reset_remain_one_page();
|
||||
}
|
||||
}
|
||||
} else {
|
||||
state_ = FINISHED;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObGlobalIndexLookupOpImpl::need_next_index_batch() const
|
||||
{
|
||||
bool bret = false;
|
||||
if (!get_batch_rescan()) {
|
||||
bret = !index_end_;
|
||||
} else if (lookup_group_cnt_ >= index_group_cnt_) {
|
||||
bret = !index_end_;
|
||||
}
|
||||
return bret;
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::check_lookup_row_cnt()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (GCONF.enable_defensive_check()
|
||||
&& get_lookup_ctdef()->pd_expr_spec_.pushdown_filters_.empty()) {
|
||||
if (OB_UNLIKELY(lookup_rowkey_cnt_ != lookup_row_cnt_)
|
||||
&& index_group_cnt_ == lookup_group_cnt_) {
|
||||
ret = OB_ERR_DEFENSIVE_CHECK;
|
||||
ObString func_name = ObString::make_string("check_lookup_row_cnt");
|
||||
LOG_USER_ERROR(OB_ERR_DEFENSIVE_CHECK, func_name.length(), func_name.ptr());
|
||||
LOG_ERROR("Fatal Error!!! Catch a defensive error!",
|
||||
K(ret), K_(lookup_rowkey_cnt), K_(lookup_row_cnt));
|
||||
//now to dump lookup das task info
|
||||
for (DASTaskIter task_iter = das_ref_.begin_task_iter(); !task_iter.is_end(); ++task_iter) {
|
||||
ObDASScanOp *das_op = static_cast<ObDASScanOp*>(*task_iter);
|
||||
LOG_INFO("dump TableLookup DAS Task range",
|
||||
"scan_range", das_op->get_scan_param().key_ranges_,
|
||||
"range_array_pos", das_op->get_scan_param().range_array_pos_);
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::do_index_table_scan_for_rows(const int64_t max_row_cnt,
|
||||
const int64_t start_group_idx,
|
||||
const int64_t default_row_batch_cnt)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const ObBatchRows* child_brs = &table_scan_op_->get_brs();
|
||||
int64_t batch_size = common::min(max_row_cnt, table_scan_op_->get_tsc_spec().max_batch_size_);
|
||||
while (OB_SUCC(ret) && lookup_rowkey_cnt_ < default_row_batch_cnt) {
|
||||
if (OB_NOT_NULL(child_brs->skip_)) {
|
||||
child_brs->skip_->reset(table_scan_op_->get_spec().max_batch_size_ > 0? table_scan_op_->get_spec().max_batch_size_ : 1);
|
||||
}
|
||||
int64_t rowkey_batch_size = min(batch_size, default_row_batch_cnt - lookup_rowkey_cnt_);
|
||||
if (OB_FAIL(table_scan_op_->inner_get_next_batch_for_tsc(rowkey_batch_size))) {
|
||||
LOG_WARN("get next row from child failed", K(ret));
|
||||
} else if (child_brs->size_ == 0 && child_brs->end_) {
|
||||
if (get_batch_rescan()) {
|
||||
if (OB_FAIL(table_scan_op_->inner_rescan_for_tsc())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("rescan index operator failed", K(ret));
|
||||
} else {
|
||||
ret = OB_SUCCESS;
|
||||
index_end_ = true;
|
||||
break;
|
||||
}
|
||||
} else {
|
||||
inc_index_group_cnt();
|
||||
LOG_DEBUG("switch to next index batch to fetch rowkey", K(index_group_cnt_), K(lookup_rowkey_cnt_));
|
||||
}
|
||||
} else {
|
||||
// index scan is finished, go to lookup stage
|
||||
index_end_ = true;
|
||||
break;
|
||||
}
|
||||
} else {
|
||||
// critical path: no child_brs sanity check
|
||||
index_end_ = (true == child_brs->end_);
|
||||
table_scan_op_->clear_evaluated_flag();
|
||||
if (OB_FAIL(process_data_table_rowkeys(child_brs->size_, child_brs->skip_))) {
|
||||
LOG_WARN("process data table rowkey with das failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
state_ = DO_LOOKUP;
|
||||
OZ(init_group_range(start_group_idx, index_group_cnt_));
|
||||
}
|
||||
LOG_DEBUG("index scan end", KPC(child_brs), K(index_end_), K(index_group_cnt_), K(lookup_rowkey_cnt_), K(lookup_group_cnt_), K(lookup_row_cnt_));
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObGlobalIndexLookupOpImpl::update_state_in_output_rows_state(int64_t &count)
|
||||
{
|
||||
UNUSED(count);
|
||||
brs_.end_ = false;
|
||||
}
|
||||
|
||||
void ObGlobalIndexLookupOpImpl::update_states_in_finish_state()
|
||||
{
|
||||
brs_.end_ = true;
|
||||
}
|
||||
|
||||
void ObGlobalIndexLookupOpImpl::reset_for_rescan()
|
||||
{
|
||||
if (lookup_memctx_ != nullptr) {
|
||||
lookup_memctx_->reset_remain_one_page();
|
||||
}
|
||||
state_ = INDEX_SCAN;
|
||||
das_ref_.reuse();
|
||||
index_end_ = false;
|
||||
index_group_cnt_ = 1;
|
||||
lookup_group_cnt_ = 1;
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::build_data_table_range(ObNewRange &lookup_range)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t rowkey_cnt = table_scan_op_->get_tsc_ctdef().global_index_rowkey_exprs_.count();
|
||||
ObObj *obj_ptr = nullptr;
|
||||
void *buf = nullptr;
|
||||
if (OB_ISNULL(buf = lookup_memctx_->get_arena_allocator().alloc(sizeof(ObObj) * rowkey_cnt))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("allocate buffer failed", K(ret), K(rowkey_cnt));
|
||||
} else {
|
||||
obj_ptr = new(buf) ObObj[rowkey_cnt];
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < rowkey_cnt; ++i) {
|
||||
ObObj tmp_obj;
|
||||
ObExpr *expr = table_scan_op_->get_tsc_ctdef().global_index_rowkey_exprs_.at(i);
|
||||
ObDatum &col_datum = expr->locate_expr_datum(table_scan_op_->get_eval_ctx());
|
||||
if (OB_FAIL(col_datum.to_obj(tmp_obj, expr->obj_meta_, expr->obj_datum_map_))) {
|
||||
LOG_WARN("convert datum to obj failed", K(ret));
|
||||
} else if (OB_FAIL(ob_write_obj(lookup_memctx_->get_arena_allocator(), tmp_obj, obj_ptr[i]))) {
|
||||
LOG_WARN("deep copy rowkey value failed", K(ret), K(tmp_obj));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
ObRowkey table_rowkey(obj_ptr, rowkey_cnt);
|
||||
uint64_t ref_table_id = get_lookup_ctdef()->ref_table_id_;
|
||||
if (OB_FAIL(lookup_range.build_range(ref_table_id, table_rowkey))) {
|
||||
LOG_WARN("build lookup range failed", K(ret), K(ref_table_id), K(table_rowkey));
|
||||
} else {
|
||||
lookup_range.group_idx_ = index_group_cnt_ - 1;
|
||||
}
|
||||
LOG_DEBUG("build data table range", K(ret), K(table_rowkey), K(lookup_range));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::switch_lookup_result_iter()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
for (DASTaskIter task_iter = das_ref_.begin_task_iter();
|
||||
OB_SUCC(ret) && !task_iter.is_end(); ++task_iter) {
|
||||
ObDASGroupScanOp *batch_op = static_cast<ObDASGroupScanOp*>(*task_iter);
|
||||
if (OB_FAIL(batch_op->switch_scan_group())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("switch batch iter failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
++lookup_group_cnt_;
|
||||
state_ = OUTPUT_ROWS;
|
||||
lookup_result_ = das_ref_.begin_result_iter();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObGlobalIndexLookupOpImpl::has_das_scan_op(const ObDASTabletLoc *tablet_loc, ObDASScanOp *&das_op)
|
||||
{
|
||||
if (get_batch_rescan()) {
|
||||
das_op = static_cast<ObDASScanOp*>(
|
||||
das_ref_.find_das_task(tablet_loc, DAS_OP_TABLE_BATCH_SCAN));
|
||||
} else {
|
||||
das_op = static_cast<ObDASScanOp*>(
|
||||
das_ref_.find_das_task(tablet_loc, DAS_OP_TABLE_SCAN));
|
||||
}
|
||||
return das_op != nullptr;
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::get_next_data_table_rows(int64_t &count,int64_t capacity)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool got_rows = false;
|
||||
if (OB_UNLIKELY(lookup_result_.is_end())) {
|
||||
ret = OB_ITER_END;
|
||||
LOG_DEBUG("lookup task is empty", K(ret));
|
||||
}
|
||||
while (OB_SUCC(ret) && !got_rows) {
|
||||
table_scan_op_->clear_evaluated_flag();
|
||||
ret = lookup_result_.get_next_rows(count, capacity);
|
||||
if (OB_ITER_END == ret && count > 0) {
|
||||
got_rows = true;
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
if (OB_ITER_END == ret) {
|
||||
if (OB_FAIL(lookup_result_.next_result())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("fetch next task result failed", K(ret));
|
||||
} else {
|
||||
// do nothing, just return OB_ITER_END to notify the caller das scan
|
||||
// reach end
|
||||
LOG_DEBUG("das_ref_ reach end, stop lookup table");
|
||||
}
|
||||
}
|
||||
} else {
|
||||
LOG_WARN("get next row from das result failed", K(ret));
|
||||
}
|
||||
} else if (count == 0) {
|
||||
if (OB_FAIL(lookup_result_.next_result())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("fetch next task failed", K(ret));
|
||||
} else {
|
||||
// do nothing, just return OB_ITER_END to notify the caller das scan
|
||||
// reach end
|
||||
LOG_DEBUG("das_ref_ reach end, stop lookup table");
|
||||
}
|
||||
}
|
||||
} else {
|
||||
got_rows = true;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && got_rows) {
|
||||
lookup_row_cnt_ += count;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGlobalIndexLookupOpImpl::reset_brs()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
brs_.size_ = 0;
|
||||
brs_.end_ = false;
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObGlobalIndexLookupOpImpl::destroy()
|
||||
{
|
||||
state_ = FINISHED;
|
||||
index_end_ = true;
|
||||
das_ref_.reset();
|
||||
if (lookup_memctx_ != nullptr) {
|
||||
lookup_memctx_->reset_remain_one_page();
|
||||
DESTROY_CONTEXT(lookup_memctx_);
|
||||
lookup_memctx_ = nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
Reference in New Issue
Block a user