[FEAT MERGE]: online optimizer stats gather.
Co-authored-by: obdev <obdev@oceanbase.com>
This commit is contained in:
@ -4846,6 +4846,15 @@ int ObAggregateProcessor::get_llc_size()
|
||||
}
|
||||
|
||||
int ObAggregateProcessor::llc_add_value(const uint64_t value, const ObString &llc_bitmap_buf)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(llc_add_value(value, const_cast<char*>(llc_bitmap_buf.ptr()), llc_bitmap_buf.length()))) {
|
||||
LOG_WARN("fail to add value", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObAggregateProcessor::llc_add_value(const uint64_t value, char *llc_bitmap_buf, int64_t size)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
uint64_t bucket_index = value >> (64 - LLC_BUCKET_BITS);
|
||||
@ -4855,10 +4864,10 @@ int ObAggregateProcessor::llc_add_value(const uint64_t value, const ObString &ll
|
||||
} else {
|
||||
pmax = ObExprEstimateNdv::llc_leading_zeros(value << LLC_BUCKET_BITS, 64 - LLC_BUCKET_BITS) + 1;
|
||||
}
|
||||
ObString::obstr_size_t llc_num_buckets = llc_bitmap_buf.length();
|
||||
if (OB_UNLIKELY(llc_bitmap_buf.length() != get_llc_size())) {
|
||||
ObString::obstr_size_t llc_num_buckets = size;
|
||||
if (OB_UNLIKELY(size != get_llc_size())) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("buffer size don't match", K(llc_bitmap_buf.length()), K(get_llc_size()));
|
||||
LOG_WARN("buffer size don't match", K(size), K(get_llc_size()));
|
||||
} else if (OB_UNLIKELY(!ObExprEstimateNdv::llc_is_num_buckets_valid(llc_num_buckets))
|
||||
|| OB_UNLIKELY(llc_num_buckets <= bucket_index)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
@ -4866,8 +4875,7 @@ int ObAggregateProcessor::llc_add_value(const uint64_t value, const ObString &ll
|
||||
K(llc_num_buckets), K(bucket_index), K(ret));
|
||||
} else if (pmax > static_cast<uint8_t>(llc_bitmap_buf[bucket_index])) {
|
||||
// 理论上pmax不会超过65.
|
||||
(const_cast<ObString &>(llc_bitmap_buf)).ptr()[bucket_index] = static_cast<uint8_t>(pmax);
|
||||
LOG_DEBUG("llc add value", K(pmax), K(bucket_index));
|
||||
llc_bitmap_buf[bucket_index] = static_cast<uint8_t>(pmax);
|
||||
}
|
||||
LOG_DEBUG("llc add value", K(pmax), K(bucket_index));
|
||||
return ret;
|
||||
|
||||
@ -665,6 +665,7 @@ public:
|
||||
int fast_single_row_agg(ObEvalCtx &eval_ctx);
|
||||
int fast_single_row_agg_batch(ObEvalCtx &eval_ctx, const int64_t batch_size, const ObBitVector *skip);
|
||||
inline void set_support_fast_single_row_agg(const bool flag) { support_fast_single_row_agg_ = flag; }
|
||||
static int llc_add_value(const uint64_t value, char *llc_bitmap_buf, int64_t size);
|
||||
private:
|
||||
template <typename T>
|
||||
int inner_process_batch(GroupRow &group_rows, T &selector, int64_t start_idx, int64_t end_idx);
|
||||
|
||||
@ -38,6 +38,7 @@
|
||||
#include "sql/code_generator/ob_expr_generator_impl.h"
|
||||
#include "sql/engine/cmd/ob_partition_executor_utils.h"
|
||||
#include "sql/parser/ob_parser.h"
|
||||
#include "share/system_variable/ob_sys_var_class_type.h"
|
||||
|
||||
#include "sql/ob_select_stmt_printer.h"
|
||||
#include "observer/ob_server_struct.h"
|
||||
@ -98,6 +99,9 @@ int ObCreateTableExecutor::prepare_ins_arg(ObCreateTableStmt &stmt,
|
||||
int64_t pos1 = 0;
|
||||
bool is_set_subquery = false;
|
||||
bool is_oracle_mode = lib::is_oracle_mode();
|
||||
bool no_osg_hint = false;
|
||||
bool osg_hint = false;
|
||||
bool online_sys_var = false;
|
||||
const ObString &db_name = stmt.get_database_name();
|
||||
const ObString &tab_name = stmt.get_table_name();
|
||||
const char sep_char = is_oracle_mode? '"': '`';
|
||||
@ -111,18 +115,37 @@ int ObCreateTableExecutor::prepare_ins_arg(ObCreateTableStmt &stmt,
|
||||
if (OB_ISNULL(buf)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_ERROR("allocate memory failed");
|
||||
} else if (OB_ISNULL(select_stmt)) {
|
||||
} else if (OB_ISNULL(select_stmt) || OB_ISNULL(select_stmt->get_query_ctx()) || OB_ISNULL(my_session)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("select stmt should not be null", K(ret));
|
||||
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos1, "insert into %c%.*s%c.%c%.*s%c",
|
||||
sep_char,
|
||||
db_name.length(),
|
||||
db_name.ptr(),
|
||||
sep_char,
|
||||
sep_char,
|
||||
tab_name.length(),
|
||||
tab_name.ptr(),
|
||||
sep_char))) {
|
||||
} else {
|
||||
//get hint
|
||||
no_osg_hint = select_stmt->get_query_ctx()->get_global_hint().has_no_gather_opt_stat_hint();
|
||||
osg_hint = select_stmt->get_query_ctx()->get_global_hint().has_gather_opt_stat_hint();
|
||||
|
||||
//get system variable
|
||||
ObObj online_sys_var_obj;
|
||||
if (OB_FAIL(OB_FAIL(my_session->get_sys_variable(SYS_VAR_ONLINE_OPT_STAT_GATHER, online_sys_var_obj)))) {
|
||||
LOG_WARN("fail to get sys var", K(ret));
|
||||
} else {
|
||||
online_sys_var = online_sys_var_obj.get_bool();
|
||||
LOG_DEBUG("online opt stat gather", K(online_sys_var), K(no_osg_hint));
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(databuff_printf(buf, buf_len, pos1,
|
||||
(!no_osg_hint && (online_sys_var || osg_hint))
|
||||
? "insert /*+GATHER_OPTIMIZER_STATISTICS*/ into %c%.*s%c.%c%.*s%c"
|
||||
: "insert into %c%.*s%c.%c%.*s%c",
|
||||
sep_char,
|
||||
db_name.length(),
|
||||
db_name.ptr(),
|
||||
sep_char,
|
||||
sep_char,
|
||||
tab_name.length(),
|
||||
tab_name.ptr(),
|
||||
sep_char))) {
|
||||
LOG_WARN("fail to print insert into string", K(ret), K(db_name), K(tab_name));
|
||||
} else if (lib::is_oracle_mode()) {
|
||||
ObTableSchema &table_schema = stmt.get_create_table_arg().schema_;
|
||||
|
||||
@ -43,8 +43,12 @@ int ObExprSysOpOpnsize::calc_sys_op_opnsize_expr(const ObExpr &expr, ObEvalCtx &
|
||||
if (OB_FAIL(expr.eval_param_value(ctx, arg))) {
|
||||
LOG_WARN("eval param failed", K(ret));
|
||||
} else {
|
||||
int64_t size = sizeof(*arg) + (arg->is_null() ? 0 : arg->len_);
|
||||
res.set_int(size);
|
||||
int64_t size = 0;
|
||||
if (OB_FAIL(calc_sys_op_opnsize(arg, size))) {
|
||||
LOG_WARN("fail to cal sys_op_opnsize", K(ret));
|
||||
} else {
|
||||
res.set_int(size);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -59,5 +63,16 @@ int ObExprSysOpOpnsize::cg_expr(ObExprCGCtx &expr_cg_ctx, const ObRawExpr &raw_e
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprSysOpOpnsize::calc_sys_op_opnsize(ObDatum *arg, int64_t &size) {
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(arg)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null pointer", K(ret));
|
||||
} else {
|
||||
size = sizeof(*arg) + (arg->is_null() ? 0 : arg->len_);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} /* namespace sql */
|
||||
} /* namespace oceanbase */
|
||||
|
||||
@ -33,6 +33,7 @@ public:
|
||||
ObExpr &rt_expr) const override;
|
||||
static int calc_sys_op_opnsize_expr(const ObExpr &expr, ObEvalCtx &ctx,
|
||||
ObDatum &res);
|
||||
static int calc_sys_op_opnsize(ObDatum *arg, int64_t &size);
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObExprSysOpOpnsize);
|
||||
};
|
||||
|
||||
@ -89,6 +89,7 @@
|
||||
#include "sql/optimizer/ob_log_insert_all.h"
|
||||
#include "sql/optimizer/ob_log_err_log.h"
|
||||
#include "sql/optimizer/ob_log_stat_collector.h"
|
||||
#include "sql/optimizer/ob_log_optimizer_stats_gathering.h"
|
||||
#include "sql/engine/aggregate/ob_merge_distinct_op.h"
|
||||
#include "sql/engine/aggregate/ob_hash_distinct_op.h"
|
||||
#include "sql/engine/basic/ob_material_op.h"
|
||||
@ -157,6 +158,7 @@
|
||||
#include "sql/engine/table/ob_link_scan_op.h"
|
||||
#include "sql/engine/dml/ob_table_insert_all_op.h"
|
||||
#include "sql/engine/basic/ob_stat_collector_op.h"
|
||||
#include "sql/engine/opt_statistics/ob_optimizer_stats_gathering_op.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
|
||||
@ -582,6 +582,12 @@ class ObStatCollectorOp;
|
||||
REGISTER_OPERATOR(ObLogStatCollector, PHY_STAT_COLLECTOR, ObStatCollectorSpec, ObStatCollectorOp, NOINPUT,
|
||||
VECTORIZED_OP);
|
||||
|
||||
class ObLogOptimizerStatsGathering;
|
||||
class ObOptimizerStatsGatheringSpec;
|
||||
class ObOptimizerStatsGatheringOp;
|
||||
REGISTER_OPERATOR(ObLogOptimizerStatsGathering, PHY_OPTIMIZER_STATS_GATHERING,
|
||||
ObOptimizerStatsGatheringSpec, ObOptimizerStatsGatheringOp, NOINPUT, VECTORIZED_OP);
|
||||
|
||||
#undef REGISTER_OPERATOR
|
||||
#undef REGISTER_OPERATOR_FULL
|
||||
#undef CHECK_IS_CHAR
|
||||
|
||||
@ -114,6 +114,7 @@ PHY_OP_DEF(PHY_PX_MULTI_PART_SSTABLE_INSERT)
|
||||
PHY_OP_DEF(PHY_ERR_LOG)
|
||||
PHY_OP_DEF(PHY_PX_ORDERED_COORD)
|
||||
PHY_OP_DEF(PHY_STAT_COLLECTOR)
|
||||
PHY_OP_DEF(PHY_OPTIMIZER_STATS_GATHERING)
|
||||
/* end of phy operator type */
|
||||
PHY_OP_DEF(PHY_NEW_OP_ADAPTER)
|
||||
PHY_OP_DEF(PHY_FAKE_TABLE) /* for testing only*/
|
||||
|
||||
@ -0,0 +1,743 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "ob_optimizer_stats_gathering_op.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/engine/px/ob_px_sqc_proxy.h"
|
||||
#include "sql/engine/px/ob_px_sqc_handler.h"
|
||||
#include "share/stat/ob_opt_stat_sql_service.h"
|
||||
#include "lib/utility/ob_unify_serialize.h"
|
||||
#include "lib/utility/ob_macro_utils.h"
|
||||
#include "sql/engine/aggregate/ob_aggregate_processor.h"
|
||||
#include "share/stat/ob_opt_stat_manager.h"
|
||||
#include "share/stat/ob_dbms_stats_executor.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
using namespace common;
|
||||
using namespace share;
|
||||
namespace sql
|
||||
{
|
||||
ObOptimizerStatsGatheringSpec::ObOptimizerStatsGatheringSpec(ObIAllocator &alloc, const ObPhyOperatorType type)
|
||||
: ObOpSpec(alloc, type),
|
||||
part_level_(schema::PARTITION_LEVEL_ZERO),
|
||||
calc_part_id_expr_(NULL),
|
||||
table_id_(OB_INVALID_ID),
|
||||
type_(OSG_TYPE::GATHER_OSG),
|
||||
target_osg_id_(OB_INVALID_ID),
|
||||
generated_column_exprs_(alloc),
|
||||
col_conv_exprs_(alloc),
|
||||
column_ids_(alloc)
|
||||
{
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringSpec::register_to_datahub(ObExecContext &ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (type_ == OSG_TYPE::GATHER_OSG) {
|
||||
if (OB_ISNULL(ctx.get_sqc_handler())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("null unexpected", K(ret));
|
||||
} else {
|
||||
void *buf = ctx.get_allocator().alloc(sizeof(ObOptStatsGatherWholeMsg::WholeMsgProvider));
|
||||
if (OB_ISNULL(buf)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("fail to allocated memory", K(ret));
|
||||
} else {
|
||||
ObOptStatsGatherWholeMsg::WholeMsgProvider *provider =
|
||||
new (buf)ObOptStatsGatherWholeMsg::WholeMsgProvider();
|
||||
ObSqcCtx &sqc_ctx = ctx.get_sqc_handler()->get_sqc_ctx();
|
||||
if (OB_FAIL(sqc_ctx.add_whole_msg_provider(get_id(), *provider))) {
|
||||
LOG_WARN("fail add whole msg provider", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObOptimizerStatsGatheringSpec, ObOpSpec),
|
||||
part_level_,
|
||||
calc_part_id_expr_,
|
||||
table_id_,
|
||||
type_,
|
||||
target_osg_id_,
|
||||
generated_column_exprs_,
|
||||
col_conv_exprs_,
|
||||
column_ids_);
|
||||
|
||||
ObOptimizerStatsGatheringOp::ObOptimizerStatsGatheringOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input)
|
||||
: ObOperator(exec_ctx, spec, input),
|
||||
tenant_id_(OB_INVALID_ID),
|
||||
table_stats_map_(),
|
||||
column_stats_map_(),
|
||||
part_map_(),
|
||||
piece_msg_(),
|
||||
arena_("ObOptStatGather")
|
||||
{
|
||||
}
|
||||
|
||||
void ObOptimizerStatsGatheringOp::destroy() {
|
||||
table_stats_map_.destroy();
|
||||
column_stats_map_.destroy();
|
||||
part_map_.destroy();
|
||||
arena_.reset();
|
||||
ObOperator::destroy();
|
||||
}
|
||||
|
||||
void ObOptimizerStatsGatheringOp::reset() {
|
||||
}
|
||||
|
||||
void ObOptimizerStatsGatheringOp::reuse_stats() {
|
||||
// call each column stat's reset to free memory allocated by inner_max_alloc and inner_min_alloc
|
||||
FOREACH(it, column_stats_map_) {
|
||||
if (OB_NOT_NULL(it->second)) {
|
||||
it->second->reset();
|
||||
it->second = NULL;
|
||||
}
|
||||
}
|
||||
table_stats_map_.reuse();
|
||||
column_stats_map_.reuse();
|
||||
arena_.reset();
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(child_) || OB_ISNULL(ctx_.get_my_session())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected status: null pointer", K(ret));
|
||||
} else {
|
||||
tenant_id_ = ctx_.get_my_session()->get_effective_tenant_id();
|
||||
arena_.set_tenant_id(tenant_id_);
|
||||
piece_msg_.set_tenant_id(tenant_id_);
|
||||
if (OB_FAIL(table_stats_map_.create(DEFAULT_HASH_MAP_BUCKETS_COUNT,
|
||||
"TabStatBucket",
|
||||
"TabStatNode"))) {
|
||||
LOG_WARN("fail to create table stats map", K(ret));
|
||||
} else if (OB_FAIL(column_stats_map_.create(DEFAULT_HASH_MAP_BUCKETS_COUNT,
|
||||
"ColStatBucket",
|
||||
"ColStatNode"))) {
|
||||
LOG_WARN("fail to create column stats map", K(ret));
|
||||
} else if (OB_FAIL(part_map_.create(DEFAULT_HASH_MAP_BUCKETS_COUNT,
|
||||
"PartMapBucket",
|
||||
"PartMapNode"))) {
|
||||
LOG_WARN("fail to create part map", K(ret));
|
||||
} else if (OB_FAIL(init_part_map())) {
|
||||
LOG_WARN("fail to init part map", K(ret));
|
||||
} else {
|
||||
// do nothing yet
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::init_part_map()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const ObTableSchema *tab_schema = nullptr;
|
||||
share::schema::ObSchemaGetterGuard *schema_guard = ctx_.get_virtual_table_ctx().schema_guard_;
|
||||
if (OB_ISNULL(schema_guard)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get schema guard", K(ret));
|
||||
} else if (OB_FAIL(schema_guard->get_table_schema(tenant_id_, MY_SPEC.table_id_, tab_schema))) {
|
||||
LOG_WARN("fail to get table schema", K(ret), K(tenant_id_), K(MY_SPEC.table_id_));
|
||||
} else if (OB_ISNULL(tab_schema)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get table schema", K(ret));
|
||||
} else {
|
||||
if (MY_SPEC.is_part_table()) {
|
||||
if (OB_FAIL(pl::ObDbmsStats::get_table_partition_map(*tab_schema, part_map_))) {
|
||||
LOG_WARN("fail to get part map", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::inner_close()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObOperator::inner_close())) {
|
||||
LOG_WARN("failed to inner close", K(ret));
|
||||
} else {
|
||||
reset();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
clear_evaluated_flag();
|
||||
if (OB_FAIL(child_->get_next_row())) {
|
||||
if (OB_ITER_END == ret) {
|
||||
if (MY_SPEC.type_ == OSG_TYPE::GATHER_OSG) {
|
||||
if (OB_FAIL(send_stats())) {
|
||||
LOG_WARN("failed to send stats", K(ret));
|
||||
}
|
||||
} else if (MY_SPEC.type_ != OSG_TYPE::GATHER_OSG) {
|
||||
if (OB_FAIL(msg_end())) {
|
||||
LOG_WARN("failed to call msg end", K(ret));
|
||||
}
|
||||
}
|
||||
reuse_stats();
|
||||
if (OB_SUCC(ret)) {
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
} else {
|
||||
LOG_WARN("fail to get next row", K(ret));
|
||||
}
|
||||
} else if (MY_SPEC.type_ != OSG_TYPE::MERGE_OSG && OB_FAIL(calc_stats())) {
|
||||
LOG_WARN("fail to calc stats", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::inner_get_next_batch(const int64_t max_row_cnt)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const int64_t batch_size = std::min(max_row_cnt, MY_SPEC.max_batch_size_);
|
||||
clear_evaluated_flag();
|
||||
const ObBatchRows *child_brs = nullptr;
|
||||
if (OB_FAIL(child_->get_next_batch(batch_size, child_brs))) {
|
||||
LOG_WARN("failed to get next batch", K(ret));
|
||||
} else if (child_brs->end_ && 0 == child_brs->size_) {
|
||||
} else if (MY_SPEC.type_ != OSG_TYPE::MERGE_OSG) {
|
||||
// set the index of output.
|
||||
ObEvalCtx::BatchInfoScopeGuard batch_info_guard(eval_ctx_);
|
||||
batch_info_guard.set_batch_size(batch_size);
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < child_brs->size_; i++) {
|
||||
if (child_brs->skip_->exist(i)) {
|
||||
} else {
|
||||
batch_info_guard.set_batch_idx(i);
|
||||
if (OB_FAIL(calc_stats())) {
|
||||
LOG_WARN("fail to calc stats", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(brs_.copy(child_brs))) {
|
||||
LOG_WARN("copy child_brs to brs_ failed", K(ret));
|
||||
} else if (brs_.end_) {
|
||||
if (MY_SPEC.type_ == OSG_TYPE::GATHER_OSG) {
|
||||
if (OB_FAIL(send_stats())) {
|
||||
LOG_WARN("failed to send stats", K(ret));
|
||||
}
|
||||
} else if (MY_SPEC.type_ != OSG_TYPE::GATHER_OSG) {
|
||||
if (OB_FAIL(msg_end())) {
|
||||
LOG_WARN("failed to call msg end", K(ret));
|
||||
}
|
||||
}
|
||||
reuse_stats();
|
||||
}
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::send_stats()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPxSqcHandler *handler = ctx_.get_sqc_handler();
|
||||
const ObOptStatsGatherWholeMsg *whole_msg = NULL;
|
||||
if (OB_ISNULL(handler)) {
|
||||
LOG_WARN("do not need sqc in serial mode");
|
||||
} else if (OB_FAIL(build_piece_msg(piece_msg_, handler->get_sqc_proxy()))) {
|
||||
LOG_WARN("failed to build piece msg", K(ret));
|
||||
} else if (OB_FAIL(handler->get_sqc_proxy().get_dh_msg(
|
||||
MY_SPEC.id_, piece_msg_, whole_msg,
|
||||
ctx_.get_physical_plan_ctx()->get_timeout_timestamp(), true, false))) {
|
||||
LOG_WARN("get msg failed", K(ret), K(MY_SPEC.id_), K(piece_msg_));
|
||||
} else {
|
||||
LOG_DEBUG("SUCCESS to send piece msg", K(ret), K(piece_msg_));
|
||||
//after send we need to reset table_stat_map and column_stat_map
|
||||
piece_msg_.reset();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::build_piece_msg(ObOptStatsGatherPieceMsg &piece,
|
||||
ObPxSQCProxy &proxy)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
piece.op_id_ = MY_SPEC.id_;
|
||||
piece.thread_id_ = GETTID();
|
||||
piece.dfo_id_ = proxy.get_dfo_id();
|
||||
piece.target_osg_id_ = MY_SPEC.target_osg_id_;
|
||||
if (OB_FAIL(get_tab_stats(piece.table_stats_))) {
|
||||
LOG_WARN("fail to get table stats", K(ret));
|
||||
} else if (OB_FAIL(get_col_stats(piece.column_stats_))) {
|
||||
LOG_WARN("fail to get column stats", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
// generate global part id, part id and subpart id(if necessary).
|
||||
int ObOptimizerStatsGatheringOp::generate_part_ids(PartIds &part_ids)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (MY_SPEC.is_part_table()) {
|
||||
ObObjectID partition_id = OB_INVALID_ID;
|
||||
ObTabletID tablet_id;
|
||||
if (OB_FAIL(ObExprCalcPartitionBase::calc_part_and_tablet_id(MY_SPEC.calc_part_id_expr_, eval_ctx_, partition_id, tablet_id))) {
|
||||
LOG_WARN("calc part and tablet id by expr failed", K(ret));
|
||||
} else {
|
||||
OSGPartInfo part_info;
|
||||
part_ids.part_id_ = partition_id;
|
||||
part_ids.global_part_id_ = -1;
|
||||
if (!MY_SPEC.is_two_level_part()) {
|
||||
} else if (OB_FAIL(part_map_.get_refactored(partition_id, part_info))) {
|
||||
if (ret != OB_HASH_NOT_EXIST) {
|
||||
LOG_WARN("fail to find hash map", K(ret));
|
||||
} else {
|
||||
LOG_WARN("fail to get first part id", K(ret), K(part_ids));
|
||||
}
|
||||
} else {
|
||||
part_ids.first_part_id_ = part_info.part_id_;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
part_ids.part_id_ = (int64_t)MY_SPEC.table_id_;
|
||||
part_ids.global_part_id_ = (int64_t)MY_SPEC.table_id_;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
||||
int ObOptimizerStatsGatheringOp::get_tab_stat_by_key(ObOptTableStat::Key &key, ObOptTableStat *&tab_stat)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
void *ptr = nullptr;
|
||||
if (OB_FAIL(table_stats_map_.get_refactored(key, tab_stat))) {
|
||||
if (OB_HASH_NOT_EXIST != ret) {
|
||||
LOG_WARN("failed to find in hashmap", K(ret));
|
||||
} else if (OB_ISNULL(ptr = arena_.alloc(sizeof(ObOptTableStat)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("memory is not enough", K(ret), K(ptr));
|
||||
} else {
|
||||
tab_stat = new (ptr) ObOptTableStat();
|
||||
tab_stat->set_table_id(MY_SPEC.table_id_);
|
||||
tab_stat->set_partition_id(key.partition_id_);
|
||||
if (OB_FAIL(table_stats_map_.set_refactored(key, tab_stat))) {
|
||||
LOG_WARN("fail to insert into hash map", K(key), KPC(tab_stat));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
LOG_DEBUG("GET tab_stat", KPC(tab_stat));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::get_col_stat_by_key(ObOptColumnStat::Key &key, ObOptColumnStat *&col_stat)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
void *ptr = nullptr;
|
||||
if(OB_FAIL(column_stats_map_.get_refactored(key, col_stat))) {
|
||||
if (OB_HASH_NOT_EXIST != ret) {
|
||||
LOG_WARN("failed to find in hashmap", K(ret));
|
||||
} else if (OB_ISNULL(ptr = arena_.alloc(sizeof(ObOptColumnStat)))){
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("memory is not enough", K(ret), K(ptr));
|
||||
} else {
|
||||
col_stat = new (ptr) ObOptColumnStat(arena_);
|
||||
col_stat->set_table_id(MY_SPEC.table_id_);
|
||||
col_stat->set_partition_id(key.partition_id_);
|
||||
col_stat->set_column_id(key.column_id_);
|
||||
if (OB_FAIL(column_stats_map_.set_refactored(key, col_stat))) {
|
||||
LOG_WARN("fail to insert into hash map", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
LOG_DEBUG("GET col_stat", KPC(col_stat));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::calc_column_stats(ObExpr *expr,
|
||||
uint64_t column_id,
|
||||
PartIds &part_ids,
|
||||
StatItems &all_stats,
|
||||
int64_t &row_len)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObDatum *datum = NULL;
|
||||
ObObj res_obj;
|
||||
if (OB_ISNULL(expr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null pointer", K(ret));
|
||||
} else if (OB_FAIL(get_col_stats_by_partinfo(part_ids, column_id, all_stats))) {
|
||||
LOG_WARN("fail to get column stat", K(ret));
|
||||
} else if (OB_FAIL(expr->eval(eval_ctx_, datum))) {
|
||||
LOG_WARN("eval expr failed", K(ret));
|
||||
} else if (OB_FAIL(datum->to_obj(res_obj, expr->obj_meta_))) {
|
||||
LOG_WARN("to obj failed", K(ret));
|
||||
} else {
|
||||
int64_t col_len = 0;
|
||||
// we set avg_len outer size the set_col_stats function
|
||||
// since derive avg_len from obj is inaccurate.
|
||||
// and we need to calc avg_len before num_not_null/num_null.
|
||||
if (OB_FAIL(ObExprSysOpOpnsize::calc_sys_op_opnsize(datum, col_len))) {
|
||||
LOG_WARN("fail to calc sys op opnsize", K(ret));
|
||||
} else if (FALSE_IT(set_col_stats_avg_len(all_stats, col_len))) {
|
||||
LOG_WARN("fail to set col_stats_avg_len", K(ret), K(col_len));
|
||||
} else if (OB_FAIL(set_col_stats(all_stats, res_obj))) {
|
||||
LOG_WARN("fail to set col stats", K(ret));
|
||||
} else {
|
||||
row_len += col_len;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::calc_stats()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
StatItems all_stats;
|
||||
PartIds part_ids;
|
||||
if (MY_SPEC.column_ids_.count() != MY_SPEC.col_conv_exprs_.count() + MY_SPEC.generated_column_exprs_.count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("column ids doesn't match the output", K(ret));
|
||||
} else if (OB_FAIL(generate_part_ids(part_ids))) {
|
||||
LOG_WARN("fail to generated part ids", K(ret));
|
||||
} else if (OB_FAIL(get_tab_stats_by_partinfo(part_ids, all_stats))) {
|
||||
LOG_WARN("fail to get table stat", K(ret));
|
||||
} else {
|
||||
int64_t row_len = 0;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < MY_SPEC.col_conv_exprs_.count(); i++) {
|
||||
uint64_t column_id = MY_SPEC.column_ids_.at(i);
|
||||
if (OB_FAIL(calc_column_stats(MY_SPEC.col_conv_exprs_.at(i), column_id, part_ids, all_stats, row_len))) {
|
||||
LOG_WARN("fail to calc column stats", K(ret));
|
||||
}
|
||||
}
|
||||
//generated column
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < MY_SPEC.generated_column_exprs_.count(); i++) {
|
||||
uint64_t column_id = MY_SPEC.column_ids_.at(i + MY_SPEC.col_conv_exprs_.count());
|
||||
if (OB_FAIL(calc_column_stats(MY_SPEC.generated_column_exprs_.at(i), column_id,
|
||||
part_ids, all_stats, row_len))) {
|
||||
LOG_WARN("fail to calc column stats", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && OB_FAIL(set_tab_stats(all_stats, row_len))) {
|
||||
LOG_WARN("fail to set col stats", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::get_tab_stats_by_partinfo(PartIds &part_ids, StatItems &all_stats)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObOptTableStat::Key global_key(tenant_id_, MY_SPEC.table_id_, part_ids.global_part_id_);
|
||||
ObOptTableStat::Key part_key(tenant_id_, MY_SPEC.table_id_, part_ids.part_id_);
|
||||
ObOptTableStat::Key first_part_key(tenant_id_, MY_SPEC.table_id_, part_ids.first_part_id_);
|
||||
if (OB_FAIL(get_tab_stat_by_key(global_key, all_stats.global_tab_stat_))) {
|
||||
LOG_WARN("fail to get global table stat", K(ret));
|
||||
} else if (MY_SPEC.is_part_table() && OB_FAIL(get_tab_stat_by_key(part_key, all_stats.part_tab_stat_))) {
|
||||
LOG_WARN("fail to get part table stat", K(ret));
|
||||
} else if (MY_SPEC.is_two_level_part() && OB_FAIL(get_tab_stat_by_key(first_part_key, all_stats.first_part_tab_stat_))) {
|
||||
LOG_WARN("fail to get first part table stat", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::get_col_stats_by_partinfo(PartIds &part_ids, uint64_t column_id, StatItems &all_stats)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObOptColumnStat::Key global_col_stats_key(tenant_id_, MY_SPEC.table_id_, part_ids.global_part_id_, column_id);
|
||||
ObOptColumnStat::Key part_col_stats_key(tenant_id_, MY_SPEC.table_id_, part_ids.part_id_, column_id);
|
||||
ObOptColumnStat::Key first_part_col_stats_key(tenant_id_, MY_SPEC.table_id_, part_ids.first_part_id_, column_id);
|
||||
if (OB_FAIL(get_col_stat_by_key(global_col_stats_key, all_stats.global_col_stat_))) {
|
||||
LOG_WARN("fail to get global table stat", K(ret));
|
||||
} else if (MY_SPEC.is_part_table() && OB_FAIL(get_col_stat_by_key(part_col_stats_key, all_stats.part_col_stat_))) {
|
||||
LOG_WARN("fail to get part table stat", K(ret));
|
||||
} else if (MY_SPEC.is_two_level_part() && OB_FAIL(get_col_stat_by_key(first_part_col_stats_key, all_stats.first_part_col_stat_))) {
|
||||
LOG_WARN("fail to get first part table stat", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObOptimizerStatsGatheringOp::set_col_stats_avg_len(StatItems &all_stats, int64_t avg_len)
|
||||
{
|
||||
all_stats.global_col_stat_->merge_avg_len(avg_len);
|
||||
if (MY_SPEC.is_part_table()) {
|
||||
all_stats.part_col_stat_->merge_avg_len(avg_len);
|
||||
}
|
||||
if (MY_SPEC.is_two_level_part()) {
|
||||
all_stats.first_part_col_stat_->merge_avg_len(avg_len);
|
||||
}
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::set_col_stats(StatItems &all_stats, ObObj &obj)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
all_stats.global_col_stat_->set_stat_level(StatLevel::TABLE_LEVEL);
|
||||
if (OB_FAIL(all_stats.global_col_stat_->merge_obj(obj))) {
|
||||
LOG_WARN("fail to set global column stat", K(ret), K(obj));
|
||||
}
|
||||
if (OB_SUCC(ret) && MY_SPEC.is_part_table()) {
|
||||
all_stats.part_col_stat_->set_stat_level(StatLevel::PARTITION_LEVEL);
|
||||
if (OB_FAIL(all_stats.part_col_stat_->merge_obj(obj))) {
|
||||
LOG_WARN("fail to set part column stat", K(ret), K(obj));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && MY_SPEC.is_two_level_part()) {
|
||||
all_stats.first_part_col_stat_->set_stat_level(StatLevel::PARTITION_LEVEL);
|
||||
all_stats.part_col_stat_->set_stat_level(StatLevel::SUBPARTITION_LEVEL);
|
||||
if (OB_FAIL(all_stats.first_part_col_stat_->merge_obj(obj))) {
|
||||
LOG_WARN("fail to set first part column stat", K(ret), K(obj));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::set_tab_stats(StatItems &all_stats, int64_t row_len)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
all_stats.global_tab_stat_->add_avg_row_size(row_len);
|
||||
all_stats.global_tab_stat_->add_row_count(1);
|
||||
all_stats.global_tab_stat_->set_object_type(StatLevel::TABLE_LEVEL);
|
||||
if (MY_SPEC.is_part_table()) {
|
||||
all_stats.part_tab_stat_->add_avg_row_size(row_len);
|
||||
all_stats.part_tab_stat_->add_row_count(1);
|
||||
all_stats.part_tab_stat_->set_object_type(StatLevel::PARTITION_LEVEL);
|
||||
}
|
||||
if (MY_SPEC.is_two_level_part()) {
|
||||
all_stats.first_part_tab_stat_->add_avg_row_size(row_len);
|
||||
all_stats.first_part_tab_stat_->add_row_count(1);
|
||||
all_stats.first_part_tab_stat_->set_object_type(StatLevel::PARTITION_LEVEL);
|
||||
all_stats.part_tab_stat_->set_object_type(StatLevel::SUBPARTITION_LEVEL);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::merge_tab_stat(ObOptTableStat *src_tab_stat)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
void *ptr = NULL;
|
||||
ObOptTableStat *tab_stat;
|
||||
ObOptTableStat::Key stat_key(tenant_id_, src_tab_stat->get_table_id(), src_tab_stat->get_partition_id());
|
||||
if (OB_FAIL(table_stats_map_.get_refactored(stat_key, tab_stat))) {
|
||||
if (OB_HASH_NOT_EXIST != ret) {
|
||||
LOG_WARN("failed to find in hashmap", K(ret));
|
||||
} else if (OB_ISNULL(ptr = arena_.alloc(sizeof(ObOptTableStat)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("memory is not enough", K(ret), K(ptr));
|
||||
} else {
|
||||
if (OB_FAIL(src_tab_stat->deep_copy((char*)ptr, sizeof(ObOptTableStat), tab_stat))) {
|
||||
LOG_WARN("fail to copy tab_stat", K(ret));
|
||||
} else if (OB_ISNULL(tab_stat)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to copy tab_stat", K(ret));
|
||||
} else if (OB_FAIL(table_stats_map_.set_refactored(stat_key, tab_stat))) {
|
||||
LOG_WARN("fail to insert stats idx to map", K(ret));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
if (OB_FAIL(tab_stat->merge_table_stat(*src_tab_stat))) {
|
||||
LOG_WARN("fail to merge two table stats", K(ret), K(tab_stat), K(src_tab_stat));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::merge_col_stat(ObOptColumnStat *src_col_stat)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
void *ptr = NULL;
|
||||
ObOptColumnStat *col_stat;
|
||||
ObOptColumnStat::Key stat_key(tenant_id_, src_col_stat->get_table_id(),
|
||||
src_col_stat->get_partition_id(), src_col_stat->get_column_id());
|
||||
if (OB_FAIL(column_stats_map_.get_refactored(stat_key, col_stat))) {
|
||||
if (OB_HASH_NOT_EXIST != ret) {
|
||||
LOG_WARN("failed to find in hashmap", K(ret));
|
||||
} else if (OB_ISNULL(ptr = arena_.alloc(src_col_stat->size()))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("memory is not enough", K(ret), K(ptr));
|
||||
} else {
|
||||
if (OB_FAIL(src_col_stat->deep_copy((char*)ptr, src_col_stat->size(), col_stat))) {
|
||||
LOG_WARN("fail to copy tab_stat", K(ret));
|
||||
} else if (OB_ISNULL(col_stat)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to copy tab_stat", K(ret));
|
||||
} else if (OB_FAIL(column_stats_map_.set_refactored(stat_key, col_stat))) {
|
||||
LOG_WARN("fail to insert stats idx to map", K(ret));
|
||||
} else {
|
||||
}
|
||||
}
|
||||
} else {
|
||||
if (OB_FAIL(col_stat->merge_column_stat(*src_col_stat))) {
|
||||
LOG_WARN("fail to merge two table stats", K(ret), K(col_stat), K(src_col_stat));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
||||
int ObOptimizerStatsGatheringOp::on_piece_msg(const ObOptStatsGatherPieceMsg &piece_msg)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
// merge table_stats and column stats.
|
||||
if (MY_SPEC.type_ != OSG_TYPE::MERGE_OSG) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("only MERGE_OSG support on_piece_msg");
|
||||
} else {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < piece_msg.table_stats_.count(); i++) {
|
||||
if (OB_FAIL(merge_tab_stat(piece_msg.table_stats_.at(i)))) {
|
||||
LOG_WARN("fail to merge table stat", K(ret));
|
||||
}
|
||||
}
|
||||
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < piece_msg.column_stats_.count(); i++) {
|
||||
if (OB_FAIL(merge_col_stat(piece_msg.column_stats_.at(i)))) {
|
||||
LOG_WARN("fail to merge column stat", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
LOG_DEBUG("OSG merge piece msg", K(piece_msg));
|
||||
piece_msg_.reset();
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::msg_end()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (MY_SPEC.type_ == OSG_TYPE::GATHER_OSG) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("gather osg shouln't reach here", K(ret));
|
||||
} else {
|
||||
share::schema::ObSchemaGetterGuard *schema_guard = nullptr;
|
||||
ObTableStatParam param;
|
||||
if (OB_ISNULL(schema_guard = ctx_.get_virtual_table_ctx().schema_guard_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null pointer", K(ret));
|
||||
} else if (OB_FAIL(generate_stat_param(param))) {
|
||||
LOG_WARN("fail to generate param", K(ret));
|
||||
} else if (OB_FAIL(ObDbmsStatsExecutor::update_stat_online(ctx_,
|
||||
param,
|
||||
schema_guard,
|
||||
get_tab_stat_map(),
|
||||
get_col_stat_map()))) {
|
||||
LOG_WARN("fail to update tab/col stats", K(ret));
|
||||
} else {}
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::generate_stat_param(ObTableStatParam ¶m)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
share::schema::ObSchemaGetterGuard *schema_guard = ctx_.get_virtual_table_ctx().schema_guard_;
|
||||
|
||||
if (OB_ISNULL(schema_guard)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get schema guard", K(ret));
|
||||
} else {
|
||||
param.tenant_id_ = tenant_id_;
|
||||
param.table_id_ = MY_SPEC.table_id_;
|
||||
param.need_global_ = true;
|
||||
param.part_level_ = MY_SPEC.part_level_;
|
||||
if (!MY_SPEC.is_part_table()) {
|
||||
param.global_part_id_ = MY_SPEC.table_id_;
|
||||
param.global_tablet_id_ = MY_SPEC.table_id_;
|
||||
param.need_part_ = false;
|
||||
param.need_subpart_ = false;
|
||||
} else {
|
||||
param.global_part_id_ = -1;
|
||||
param.global_tablet_id_ = -1;
|
||||
}
|
||||
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < MY_SPEC.column_ids_.count(); i++) {
|
||||
ObColumnStatParam col_param;
|
||||
col_param.column_id_ = MY_SPEC.column_ids_.at(i);
|
||||
const ObColumnSchemaV2 *col_schema = nullptr;
|
||||
if (OB_FAIL(schema_guard->get_column_schema(tenant_id_, MY_SPEC.table_id_, col_param.column_id_, col_schema))) {
|
||||
LOG_WARN("can't get column schema", K(ret), K(tenant_id_), K(MY_SPEC.table_id_), K(col_param.column_id_));
|
||||
} else if (OB_ISNULL(col_schema)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("can't get column schema", K(ret), K(tenant_id_), K(MY_SPEC.table_id_), K(col_param.column_id_));
|
||||
} else {
|
||||
col_param.cs_type_ = col_schema->get_collation_type();
|
||||
}
|
||||
if (OB_SUCC(ret) && OB_FAIL(param.column_params_.push_back(col_param))) {
|
||||
LOG_WARN("fail to push back column param", K(ret));
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (MY_SPEC.is_part_table() && !MY_SPEC.is_two_level_part()) {
|
||||
param.need_part_ = true;
|
||||
param.need_subpart_ = false;
|
||||
} else if (MY_SPEC.is_part_table() && MY_SPEC.is_two_level_part()){
|
||||
//default is true
|
||||
}
|
||||
if (OB_SUCC(ret) && MY_SPEC.is_part_table()) {
|
||||
FOREACH_X(it, part_map_, OB_SUCC(ret)) {
|
||||
PartInfo tmp_part_info;
|
||||
tmp_part_info.part_id_ = it->first;
|
||||
tmp_part_info.tablet_id_ = it->second.tablet_id_;
|
||||
if (it->first != it->second.part_id_) {
|
||||
//subpart
|
||||
if (OB_FAIL(param.subpart_infos_.push_back(tmp_part_info))) {
|
||||
LOG_WARN("fail to push back part info", K(ret));
|
||||
}
|
||||
} else {
|
||||
//first level parttion
|
||||
if (OB_FAIL(param.part_infos_.push_back(tmp_part_info))) {
|
||||
LOG_WARN("fail to push back part info", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::get_col_stats(common::ObIArray<ObOptColumnStat*>& col_stats)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
FOREACH_X(it, column_stats_map_, OB_SUCC(ret)) {
|
||||
if (OB_ISNULL(it->second)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null pointer", K(ret));
|
||||
} else if (OB_FAIL(col_stats.push_back(it->second))) {
|
||||
LOG_WARN("fail to push back col stats", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerStatsGatheringOp::get_tab_stats(common::ObIArray<ObOptTableStat*>& tab_stats)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
FOREACH_X(it, table_stats_map_, OB_SUCC(ret)) {
|
||||
if (OB_ISNULL(it->second)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null pointer", K(ret));
|
||||
} else if (OB_FAIL(tab_stats.push_back(it->second))) {
|
||||
LOG_WARN("fail to push back col stats", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
168
src/sql/engine/opt_statistics/ob_optimizer_stats_gathering_op.h
Normal file
168
src/sql/engine/opt_statistics/ob_optimizer_stats_gathering_op.h
Normal file
@ -0,0 +1,168 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef SRC_SQL_ENGINE_BASIC_OB_OPTIMIZER_STATS_GATHERING_OP_H_
|
||||
#define SRC_SQL_ENGINE_BASIC_OB_OPTIMIZER_STATS_GATHERING_OP_H_
|
||||
#include "sql/engine/ob_operator.h"
|
||||
#include "share/datum/ob_datum.h"
|
||||
#include "sql/engine/expr/ob_expr_sys_op_opnsize.h"
|
||||
#include "lib/hash/ob_hashmap.h"
|
||||
#include "sql/engine/px/datahub/components/ob_dh_opt_stats_gather.h"
|
||||
#include "share/stat/ob_opt_table_stat.h"
|
||||
#include "share/stat/ob_opt_column_stat.h"
|
||||
#include "sql/engine/px/ob_px_sqc_handler.h"
|
||||
#include "share/stat/ob_basic_stats_estimator.h"
|
||||
#include "share/stat/ob_dbms_stats_executor.h"
|
||||
#include "share/stat/ob_stat_define.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace sql
|
||||
{
|
||||
|
||||
class ObOptimizerStatsGatheringSpec : public ObOpSpec
|
||||
{
|
||||
OB_UNIS_VERSION_V(1);
|
||||
public:
|
||||
|
||||
ObOptimizerStatsGatheringSpec(common::ObIAllocator &alloc, const ObPhyOperatorType type);
|
||||
int register_to_datahub(ObExecContext &ctx) const;
|
||||
inline void set_osg_type(OSG_TYPE type) {type_ = type; };
|
||||
inline void set_target_osg_id(uint64_t target_id) { target_osg_id_ = target_id; };
|
||||
inline bool is_part_table() const {
|
||||
return part_level_ == share::schema::PARTITION_LEVEL_ONE ||
|
||||
part_level_ == share::schema::PARTITION_LEVEL_TWO;
|
||||
};
|
||||
inline bool is_two_level_part() const {
|
||||
return part_level_ == share::schema::PARTITION_LEVEL_TWO;
|
||||
}
|
||||
|
||||
ObPartitionLevel part_level_;
|
||||
ObExpr *calc_part_id_expr_;
|
||||
uint64_t table_id_;
|
||||
OSG_TYPE type_;
|
||||
uint64_t target_osg_id_;
|
||||
ExprFixedArray generated_column_exprs_;
|
||||
ExprFixedArray col_conv_exprs_;
|
||||
ObFixedArray<uint64_t, common::ObIAllocator> column_ids_;
|
||||
};
|
||||
|
||||
class ObOptimizerStatsGatheringOp : public ObOperator
|
||||
{
|
||||
public:
|
||||
|
||||
// store global/part/subpart part_id.
|
||||
struct PartIds {
|
||||
PartIds() : global_part_id_(common::OB_INVALID_ID),
|
||||
part_id_(common::OB_INVALID_ID),
|
||||
first_part_id_(common::OB_INVALID_ID) {};
|
||||
ObObjectID global_part_id_;
|
||||
ObObjectID part_id_;
|
||||
ObObjectID first_part_id_; // for two_level partition.
|
||||
|
||||
TO_STRING_KV(K(global_part_id_), K(part_id_), K(first_part_id_));
|
||||
};
|
||||
|
||||
// store global/part/subpart table/column stat.
|
||||
struct StatItems {
|
||||
StatItems () : global_tab_stat_(nullptr),
|
||||
part_tab_stat_(nullptr),
|
||||
first_part_tab_stat_(nullptr),
|
||||
global_col_stat_(nullptr),
|
||||
part_col_stat_(nullptr),
|
||||
first_part_col_stat_(nullptr) {}
|
||||
|
||||
ObOptTableStat *global_tab_stat_;
|
||||
ObOptTableStat *part_tab_stat_;
|
||||
ObOptTableStat *first_part_tab_stat_;
|
||||
ObOptColumnStat *global_col_stat_;
|
||||
ObOptColumnStat *part_col_stat_;
|
||||
ObOptColumnStat *first_part_col_stat_;
|
||||
};
|
||||
|
||||
public:
|
||||
ObOptimizerStatsGatheringOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input);
|
||||
virtual int inner_open() override;
|
||||
virtual int inner_close() override;
|
||||
// do not need to overwitten rescan(), since this operator may have part-gi above.
|
||||
virtual int inner_get_next_row() override;
|
||||
virtual int inner_get_next_batch(const int64_t max_row_cnt) override;
|
||||
virtual void destroy() override;
|
||||
|
||||
virtual void reset();
|
||||
// will be called from datahub, when qc/sqc finish gathering stats, the function will be called in dh.
|
||||
int on_piece_msg(const ObOptStatsGatherPieceMsg &piece_msg);
|
||||
// after dh receive all piece_msg, msg_end will be called.
|
||||
int msg_end();
|
||||
|
||||
private:
|
||||
static const int64_t DEFAULT_HASH_MAP_BUCKETS_COUNT = 100;
|
||||
void reuse_stats();
|
||||
inline TabStatIndMap& get_tab_stat_map() { return table_stats_map_;};
|
||||
inline ColStatIndMap& get_col_stat_map() { return column_stats_map_;};
|
||||
|
||||
// set piece_msg's basic info, copy tab/column stats from stat_map to piece_msg's array.
|
||||
int build_piece_msg(ObOptStatsGatherPieceMsg &piece,
|
||||
ObPxSQCProxy &proxy);
|
||||
|
||||
// send piece_msg, no need to wait the whole msg.
|
||||
int send_stats();
|
||||
int calc_stats();
|
||||
// calc stats for each column
|
||||
int calc_column_stats(ObExpr *expr,
|
||||
uint64_t column_id,
|
||||
PartIds &part_ids,
|
||||
StatItems &all_stats,
|
||||
int64_t &row_len);
|
||||
int init_part_map();
|
||||
// generate stat_param that is used to write inner_table.
|
||||
int generate_stat_param(ObTableStatParam ¶m);
|
||||
int generate_part_ids(PartIds &part_ids);
|
||||
|
||||
int get_tab_stats_by_partinfo(PartIds &part_ids, StatItems &stat_item);
|
||||
int get_col_stats_by_partinfo(PartIds &part_ids, uint64_t column_id, StatItems &stat_item);
|
||||
|
||||
// get stat by part_ids
|
||||
int set_col_stats(StatItems &all_stat, common::ObObj &obj);
|
||||
void set_col_stats_avg_len(StatItems &all_stats, int64_t avg_len);
|
||||
int set_tab_stats(StatItems &all_stat, int64_t row_len);
|
||||
|
||||
// get tab stat by key(tenant_id, table_id, partition_id), if NOT_EXISTS, alloc a new one.
|
||||
int get_tab_stat_by_key(ObOptTableStat::Key &key, ObOptTableStat *&tab_stat);
|
||||
// get tab stat by key(tenant_id, table_id, partition_id, column_id), if NOT_EXISTS, alloc a new one.
|
||||
int get_col_stat_by_key(ObOptColumnStat::Key &key, ObOptColumnStat *&col_stat);
|
||||
|
||||
int merge_tab_stat(ObOptTableStat *src_tab_stat);
|
||||
int merge_col_stat(ObOptColumnStat *src_col_stat);
|
||||
|
||||
// get stats array from stats map
|
||||
int get_tab_stats(common::ObIArray<ObOptTableStat*>& tab_stats);
|
||||
int get_col_stats(common::ObIArray<ObOptColumnStat*>& col_stats);
|
||||
|
||||
uint64_t tenant_id_;
|
||||
|
||||
TabStatIndMap table_stats_map_;
|
||||
ColStatIndMap column_stats_map_;
|
||||
|
||||
// store all part information:
|
||||
// 1. all part/subpart ids
|
||||
// 2. the map between subpart_id and part_id.
|
||||
// 3. the map tetween part_id and tablet_id.
|
||||
OSGPartMap part_map_;
|
||||
|
||||
// ------- data struct for piece msg;
|
||||
ObOptStatsGatherPieceMsg piece_msg_;
|
||||
ObArenaAllocator arena_;
|
||||
};
|
||||
}
|
||||
}
|
||||
#endif /* SRC_SQL_ENGINE_BASIC_OB_MONITORING_DUMP_OP_H_ */
|
||||
156
src/sql/engine/px/datahub/components/ob_dh_opt_stats_gather.cpp
Normal file
156
src/sql/engine/px/datahub/components/ob_dh_opt_stats_gather.cpp
Normal file
@ -0,0 +1,156 @@
|
||||
// Copyright 1999-2022 Alibaba Inc. All Rights Reserved.
|
||||
// Author:
|
||||
// liuqifan.lqf@oceanbase.com
|
||||
//
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "sql/engine/px/datahub/components/ob_dh_opt_stats_gather.h"
|
||||
#include "sql/engine/px/datahub/ob_dh_msg_ctx.h"
|
||||
#include "sql/engine/px/ob_dfo.h"
|
||||
#include "sql/engine/px/ob_px_util.h"
|
||||
#include "sql/engine/px/datahub/ob_dh_msg.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/engine/opt_statistics/ob_optimizer_stats_gathering_op.h"
|
||||
|
||||
using namespace oceanbase::sql;
|
||||
using namespace oceanbase::common;
|
||||
|
||||
OB_DEF_SERIALIZE(ObOptStatsGatherPieceMsg)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ret = ObDatahubPieceMsg::serialize(buf, buf_len, pos);
|
||||
if (OB_SUCC(ret)) {
|
||||
OB_UNIS_ENCODE(table_stats_.count());
|
||||
for (int64_t i = 0; i < table_stats_.count(); i++) {
|
||||
OB_UNIS_ENCODE(*table_stats_.at(i));
|
||||
}
|
||||
OB_UNIS_ENCODE(column_stats_.count());
|
||||
for (int64_t i = 0; i < column_stats_.count(); i++) {
|
||||
OB_UNIS_ENCODE(column_stats_.at(i)->size());
|
||||
OB_UNIS_ENCODE(*column_stats_.at(i));
|
||||
}
|
||||
OB_UNIS_ENCODE(target_osg_id_);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_DESERIALIZE(ObOptStatsGatherPieceMsg)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ret = ObDatahubPieceMsg::deserialize(buf, data_len, pos);
|
||||
if (OB_SUCC(ret)) {
|
||||
int64_t size = 0;
|
||||
OB_UNIS_DECODE(size)
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < size; ++i) {
|
||||
void *tmp_buf = arena_.alloc(sizeof(ObOptTableStat));
|
||||
if (OB_ISNULL(tmp_buf)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("allocate memory failed", K(ret));
|
||||
} else {
|
||||
ObOptTableStat *tmp_stat = new (tmp_buf) ObOptTableStat;
|
||||
if (OB_FAIL(tmp_stat->deserialize(buf, data_len, pos))) {
|
||||
LOG_WARN("deserialize datum store failed", K(ret), K(i));
|
||||
} else if (OB_FAIL(table_stats_.push_back(tmp_stat))) {
|
||||
LOG_WARN("push back datum store failed", K(ret), K(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
size = 0;
|
||||
OB_UNIS_DECODE(size)
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < size; ++i) {
|
||||
int col_stat_size = 0;
|
||||
OB_UNIS_DECODE(col_stat_size);
|
||||
void *tmp_buf = arena_.alloc(col_stat_size);
|
||||
if (OB_ISNULL(tmp_buf)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("allocate memory failed", K(ret));
|
||||
} else {
|
||||
ObOptColumnStat *tmp_stat = new (tmp_buf) ObOptColumnStat(arena_);
|
||||
if (OB_FAIL(tmp_stat->deserialize(buf, data_len, pos))) {
|
||||
LOG_WARN("deserialize datum store failed", K(ret), K(i));
|
||||
} else if (OB_FAIL(column_stats_.push_back(tmp_stat))) {
|
||||
LOG_WARN("push back datum store failed", K(ret), K(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
OB_UNIS_DECODE(target_osg_id_);
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE_SIZE(ObOptStatsGatherPieceMsg)
|
||||
{
|
||||
int64_t len = 0;
|
||||
len += ObDatahubPieceMsg::get_serialize_size();
|
||||
OB_UNIS_ADD_LEN(table_stats_.count());
|
||||
for (int64_t i = 0; i < table_stats_.count(); i++) {
|
||||
OB_UNIS_ADD_LEN(*table_stats_.at(i));
|
||||
}
|
||||
OB_UNIS_ADD_LEN(column_stats_.count());
|
||||
for (int64_t i = 0; i < column_stats_.count(); i++) {
|
||||
OB_UNIS_ADD_LEN(column_stats_.at(i)->size());
|
||||
OB_UNIS_ADD_LEN(*column_stats_.at(i));
|
||||
}
|
||||
OB_UNIS_ADD_LEN(target_osg_id_);
|
||||
return len;
|
||||
}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObOptStatsGatherWholeMsg, ObDatahubWholeMsg), ready_state_);
|
||||
|
||||
int ObOptStatsGatherPieceMsgListener::on_message(
|
||||
ObOptStatsGatherPieceMsgCtx &piece_ctx,
|
||||
common::ObIArray<ObPxSqcMeta *> &sqcs,
|
||||
const ObOptStatsGatherPieceMsg &pkt)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (pkt.op_id_ != piece_ctx.op_id_) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected piece msg", K(ret), K(pkt), K(piece_ctx));
|
||||
} else {
|
||||
ObOptimizerStatsGatheringOp *osg_op = NULL;
|
||||
piece_ctx.received_++;
|
||||
// get the merge osg, pass piece msg to osg.
|
||||
ObOperatorKit *kit = piece_ctx.osg_info_.op_kit_;
|
||||
if (OB_ISNULL(kit) || OB_ISNULL(kit->op_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get null op kit", K(ret), K(kit));
|
||||
} else if (OB_UNLIKELY(PHY_OPTIMIZER_STATS_GATHERING != kit->op_->get_spec().type_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected op", K(ret), K(kit->op_->get_spec().type_));
|
||||
} else {
|
||||
osg_op = static_cast<ObOptimizerStatsGatheringOp *>(kit->op_);
|
||||
if (OB_FAIL(osg_op->on_piece_msg(pkt))) {
|
||||
LOG_WARN("fail to call on piece msg", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptStatsGatherPieceMsgCtx::alloc_piece_msg_ctx(const ObOptStatsGatherPieceMsg &pkt,
|
||||
ObPxCoordInfo &,
|
||||
ObExecContext &ctx,
|
||||
int64_t task_cnt,
|
||||
ObPieceMsgCtx *&msg_ctx)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
void *buf = ctx.get_allocator().alloc(sizeof(ObOptStatsGatherPieceMsgCtx));
|
||||
if (OB_ISNULL(ctx.get_physical_plan_ctx())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("physical plan ctx is null", K(ret));
|
||||
} else if (OB_ISNULL(buf)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
} else {
|
||||
msg_ctx = new (buf) ObOptStatsGatherPieceMsgCtx(pkt.op_id_, task_cnt,
|
||||
ctx.get_physical_plan_ctx()->get_timeout_timestamp());
|
||||
if (OB_ISNULL(ctx.get_operator_kit(pkt.target_osg_id_))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get null operator kit", K(ret), K(pkt));
|
||||
} else {
|
||||
static_cast<ObOptStatsGatherPieceMsgCtx*>(msg_ctx)->osg_info_.op_kit_ = ctx.get_operator_kit(pkt.target_osg_id_);
|
||||
}
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
137
src/sql/engine/px/datahub/components/ob_dh_opt_stats_gather.h
Normal file
137
src/sql/engine/px/datahub/components/ob_dh_opt_stats_gather.h
Normal file
@ -0,0 +1,137 @@
|
||||
// Copyright 1999-2022 Alibaba Inc. All Rights Reserved.
|
||||
// Author:
|
||||
// liuqifan.lqf@oceanbase.com
|
||||
//
|
||||
|
||||
#ifndef __OB_SQL_ENG_PX_DH_OPT_STATS_GATHER_H__
|
||||
#define __OB_SQL_ENG_PX_DH_OPT_STATS_GATHER_H__
|
||||
|
||||
#include "sql/engine/px/datahub/ob_dh_msg.h"
|
||||
#include "sql/engine/px/datahub/ob_dh_dtl_proc.h"
|
||||
#include "sql/engine/px/datahub/ob_dh_msg_ctx.h"
|
||||
#include "sql/engine/px/datahub/ob_dh_msg_provider.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
class ObOptStatsGatherPieceMsg;
|
||||
class ObOptStatsGatherWholeMsg;
|
||||
typedef ObPieceMsgP<ObOptStatsGatherPieceMsg> ObOptStatsGatherPieceMsgP;
|
||||
typedef ObWholeMsgP<ObOptStatsGatherWholeMsg> ObOptStatsGatherWholeMsgP;
|
||||
class ObOptStatsGatherPieceMsgListener;
|
||||
class ObOptStatsGatherPieceMsgCtx;
|
||||
class ObPxCoordInfo;
|
||||
|
||||
struct OSGInfo {
|
||||
OSGInfo() = default;
|
||||
~OSGInfo() = default;
|
||||
ObOperatorKit *op_kit_;
|
||||
};
|
||||
|
||||
class ObOptStatsGatherPieceMsg
|
||||
: public ObDatahubPieceMsg<dtl::ObDtlMsgType::DH_OPT_STATS_GATHER_PIECE_MSG>
|
||||
{
|
||||
OB_UNIS_VERSION_V(1);
|
||||
public:
|
||||
using PieceMsgListener = ObOptStatsGatherPieceMsgListener;
|
||||
using PieceMsgCtx = ObOptStatsGatherPieceMsgCtx;
|
||||
public:
|
||||
ObOptStatsGatherPieceMsg() :
|
||||
column_stats_(),
|
||||
table_stats_(),
|
||||
arena_("ObOSGPieceMsg"),
|
||||
target_osg_id_(0) {};
|
||||
~ObOptStatsGatherPieceMsg() { reset(); };
|
||||
void reset() {
|
||||
for (int64_t i = 0; i < column_stats_.count(); i++) {
|
||||
if (OB_NOT_NULL(column_stats_.at(i))) {
|
||||
// to clear memory allocate by inner_alloc in ObOptColumnStat
|
||||
column_stats_.at(i)->reset();
|
||||
column_stats_.at(i) = NULL;
|
||||
}
|
||||
}
|
||||
column_stats_.reset();
|
||||
table_stats_.reset();
|
||||
arena_.reset();
|
||||
};
|
||||
int assign(const ObOptStatsGatherPieceMsg &other) {
|
||||
// do assign
|
||||
return common::OB_SUCCESS;
|
||||
}
|
||||
INHERIT_TO_STRING_KV("meta", ObDatahubPieceMsg<dtl::ObDtlMsgType::DH_OPT_STATS_GATHER_PIECE_MSG>,
|
||||
K_(op_id),
|
||||
K_(column_stats),
|
||||
K_(table_stats),
|
||||
K_(target_osg_id));
|
||||
inline void set_osg_id(uint64_t target_id) { target_osg_id_ = target_id; };
|
||||
inline uint64_t get_osg_id() { return target_osg_id_; };
|
||||
|
||||
inline void set_tenant_id(uint64_t tenant_id) { arena_.set_tenant_id(tenant_id); };
|
||||
common::ObSEArray<ObOptColumnStat*, 4> column_stats_;
|
||||
common::ObSEArray<ObOptTableStat*, 4> table_stats_;
|
||||
|
||||
// to restore history
|
||||
ObArenaAllocator arena_;
|
||||
uint64_t target_osg_id_;
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObOptStatsGatherPieceMsg);
|
||||
};
|
||||
|
||||
// whole msg is useless, since we don't need the feed back. px only pass the piece msg to osg.
|
||||
class ObOptStatsGatherWholeMsg
|
||||
: public ObDatahubWholeMsg<dtl::ObDtlMsgType::DH_OPT_STATS_GATHER_WHOLE_MSG>
|
||||
{
|
||||
OB_UNIS_VERSION_V(1);
|
||||
public:
|
||||
using WholeMsgProvider = ObWholeMsgProvider<ObOptStatsGatherWholeMsg>;
|
||||
public:
|
||||
ObOptStatsGatherWholeMsg() : ready_state_(0) {};
|
||||
~ObOptStatsGatherWholeMsg() = default;
|
||||
int assign(const ObOptStatsGatherWholeMsg &other)
|
||||
{
|
||||
ready_state_ = other.ready_state_;
|
||||
return common::OB_SUCCESS;
|
||||
}
|
||||
void reset()
|
||||
{
|
||||
ready_state_ = 0;
|
||||
}
|
||||
VIRTUAL_TO_STRING_KV(K_(ready_state));
|
||||
int ready_state_;
|
||||
};
|
||||
|
||||
class ObOptStatsGatherPieceMsgCtx : public ObPieceMsgCtx
|
||||
{
|
||||
public:
|
||||
ObOptStatsGatherPieceMsgCtx(uint64_t op_id, int64_t task_cnt, int64_t timeout_ts)
|
||||
: ObPieceMsgCtx(op_id, task_cnt, timeout_ts), received_(0), osg_info_() {}
|
||||
~ObOptStatsGatherPieceMsgCtx() = default;
|
||||
static int alloc_piece_msg_ctx(const ObOptStatsGatherPieceMsg &pkt,
|
||||
ObPxCoordInfo &coord_info,
|
||||
ObExecContext &ctx,
|
||||
int64_t task_cnt,
|
||||
ObPieceMsgCtx *&msg_ctx);
|
||||
INHERIT_TO_STRING_KV("meta", ObPieceMsgCtx, K_(received));
|
||||
int received_;
|
||||
OSGInfo osg_info_; // store the op_kit for OSG_MERGE. So that we can call API in OSG.
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObOptStatsGatherPieceMsgCtx);
|
||||
};
|
||||
|
||||
class ObOptStatsGatherPieceMsgListener
|
||||
{
|
||||
public:
|
||||
ObOptStatsGatherPieceMsgListener() = default;
|
||||
~ObOptStatsGatherPieceMsgListener() = default;
|
||||
static int on_message(
|
||||
ObOptStatsGatherPieceMsgCtx &piece_ctx,
|
||||
common::ObIArray<ObPxSqcMeta *> &sqcs,
|
||||
const ObOptStatsGatherPieceMsg &pkt);
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObOptStatsGatherPieceMsgListener);
|
||||
};
|
||||
|
||||
}
|
||||
}
|
||||
#endif
|
||||
@ -48,7 +48,8 @@ ObPxFifoCoordOp::ObPxFifoCoordOp(ObExecContext &exec_ctx, const ObOpSpec &spec,
|
||||
interrupt_proc_(exec_ctx, msg_proc_),
|
||||
sample_piece_msg_proc_(exec_ctx, msg_proc_),
|
||||
rollup_key_piece_msg_proc_(exec_ctx, msg_proc_),
|
||||
rd_wf_piece_msg_proc_(exec_ctx, msg_proc_)
|
||||
rd_wf_piece_msg_proc_(exec_ctx, msg_proc_),
|
||||
opt_stats_gather_piece_msg_proc_(exec_ctx, msg_proc_)
|
||||
{}
|
||||
|
||||
int ObPxFifoCoordOp::inner_open()
|
||||
@ -96,6 +97,7 @@ int ObPxFifoCoordOp::setup_loop_proc()
|
||||
.register_processor(sample_piece_msg_proc_)
|
||||
.register_processor(rollup_key_piece_msg_proc_)
|
||||
.register_processor(rd_wf_piece_msg_proc_)
|
||||
.register_processor(opt_stats_gather_piece_msg_proc_)
|
||||
.register_interrupt_processor(interrupt_proc_);
|
||||
return ret;
|
||||
}
|
||||
@ -205,6 +207,7 @@ int ObPxFifoCoordOp::fetch_rows(const int64_t row_cnt)
|
||||
case ObDtlMsgType::DH_DYNAMIC_SAMPLE_PIECE_MSG:
|
||||
case ObDtlMsgType::DH_ROLLUP_KEY_PIECE_MSG:
|
||||
case ObDtlMsgType::DH_RANGE_DIST_WF_PIECE_MSG:
|
||||
case ObDtlMsgType::DH_OPT_STATS_GATHER_PIECE_MSG:
|
||||
// all message processed in callback
|
||||
break;
|
||||
default:
|
||||
|
||||
@ -101,6 +101,7 @@ private:
|
||||
ObDynamicSamplePieceMsgP sample_piece_msg_proc_;
|
||||
ObRollupKeyPieceMsgP rollup_key_piece_msg_proc_;
|
||||
ObRDWFPieceMsgP rd_wf_piece_msg_proc_;
|
||||
ObOptStatsGatherPieceMsgP opt_stats_gather_piece_msg_proc_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
|
||||
@ -94,6 +94,7 @@ ObPxMSCoordOp::ObPxMSCoordOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOp
|
||||
sample_piece_msg_proc_(exec_ctx, msg_proc_),
|
||||
rollup_key_piece_msg_proc_(exec_ctx, msg_proc_),
|
||||
rd_wf_piece_msg_proc_(exec_ctx, msg_proc_),
|
||||
opt_stats_gather_piece_msg_proc_(exec_ctx, msg_proc_),
|
||||
store_rows_(),
|
||||
last_pop_row_(nullptr),
|
||||
row_heap_(),
|
||||
@ -168,6 +169,7 @@ int ObPxMSCoordOp::setup_loop_proc()
|
||||
.register_processor(sample_piece_msg_proc_)
|
||||
.register_processor(rollup_key_piece_msg_proc_)
|
||||
.register_processor(rd_wf_piece_msg_proc_)
|
||||
.register_processor(opt_stats_gather_piece_msg_proc_)
|
||||
.register_interrupt_processor(interrupt_proc_);
|
||||
msg_loop_.set_tenant_id(ctx_.get_my_session()->get_effective_tenant_id());
|
||||
return ret;
|
||||
@ -407,6 +409,7 @@ int ObPxMSCoordOp::inner_get_next_row()
|
||||
case ObDtlMsgType::DH_DYNAMIC_SAMPLE_PIECE_MSG:
|
||||
case ObDtlMsgType::DH_ROLLUP_KEY_PIECE_MSG:
|
||||
case ObDtlMsgType::DH_RANGE_DIST_WF_PIECE_MSG:
|
||||
case ObDtlMsgType::DH_OPT_STATS_GATHER_PIECE_MSG:
|
||||
// 这几种消息都在 process 回调函数里处理了
|
||||
break;
|
||||
default:
|
||||
|
||||
@ -156,6 +156,7 @@ private:
|
||||
ObDynamicSamplePieceMsgP sample_piece_msg_proc_;
|
||||
ObRollupKeyPieceMsgP rollup_key_piece_msg_proc_;
|
||||
ObRDWFPieceMsgP rd_wf_piece_msg_proc_;
|
||||
ObOptStatsGatherPieceMsgP opt_stats_gather_piece_msg_proc_;
|
||||
// 存储merge sort的每一路的当前行
|
||||
ObArray<ObChunkDatumStore::LastStoredRow*> store_rows_;
|
||||
ObChunkDatumStore::LastStoredRow* last_pop_row_;
|
||||
|
||||
@ -50,6 +50,7 @@ ObPxOrderedCoordOp::ObPxOrderedCoordOp(ObExecContext &exec_ctx, const ObOpSpec &
|
||||
sample_piece_msg_proc_(exec_ctx, msg_proc_),
|
||||
rollup_key_piece_msg_proc_(exec_ctx, msg_proc_),
|
||||
rd_wf_piece_msg_proc_(exec_ctx, msg_proc_),
|
||||
opt_stats_gather_piece_msg_proc_(exec_ctx, msg_proc_),
|
||||
readers_(NULL),
|
||||
receive_order_(),
|
||||
reader_cnt_(0),
|
||||
@ -110,6 +111,7 @@ int ObPxOrderedCoordOp::setup_loop_proc()
|
||||
.register_processor(sample_piece_msg_proc_)
|
||||
.register_processor(rollup_key_piece_msg_proc_)
|
||||
.register_processor(rd_wf_piece_msg_proc_)
|
||||
.register_processor(opt_stats_gather_piece_msg_proc_)
|
||||
.register_interrupt_processor(interrupt_proc_);
|
||||
return ret;
|
||||
}
|
||||
@ -204,6 +206,7 @@ int ObPxOrderedCoordOp::inner_get_next_row()
|
||||
case ObDtlMsgType::DH_DYNAMIC_SAMPLE_PIECE_MSG:
|
||||
case ObDtlMsgType::DH_ROLLUP_KEY_PIECE_MSG:
|
||||
case ObDtlMsgType::DH_RANGE_DIST_WF_PIECE_MSG:
|
||||
case ObDtlMsgType::DH_OPT_STATS_GATHER_PIECE_MSG:
|
||||
// 这几种消息都在 process 回调函数里处理了
|
||||
break;
|
||||
default:
|
||||
|
||||
@ -139,6 +139,7 @@ private:
|
||||
ObDynamicSamplePieceMsgP sample_piece_msg_proc_;
|
||||
ObRollupKeyPieceMsgP rollup_key_piece_msg_proc_;
|
||||
ObRDWFPieceMsgP rd_wf_piece_msg_proc_;
|
||||
ObOptStatsGatherPieceMsgP opt_stats_gather_piece_msg_proc_;
|
||||
ObReceiveRowReader *readers_;
|
||||
ObOrderedReceiveFilter receive_order_;
|
||||
int64_t reader_cnt_;
|
||||
|
||||
@ -125,3 +125,10 @@ int ObPxSubCoordMsgProc::on_whole_msg(
|
||||
ObDhWholeeMsgProc<ObRDWFWholeMsg> proc;
|
||||
return proc.on_whole_msg(sqc_ctx_, pkt);
|
||||
}
|
||||
|
||||
int ObPxSubCoordMsgProc::on_whole_msg(
|
||||
const ObOptStatsGatherWholeMsg &pkt) const
|
||||
{
|
||||
ObDhWholeeMsgProc<ObOptStatsGatherWholeMsg> proc;
|
||||
return proc.on_whole_msg(sqc_ctx_, pkt);
|
||||
}
|
||||
@ -31,6 +31,8 @@ class ObDynamicSamplePieceMsg;
|
||||
class ObDynamicSampleWholeMsg;
|
||||
class ObRDWFPieceMsg;
|
||||
class ObRDWFWholeMsg;
|
||||
class ObOptStatsGatherPieceMsg;
|
||||
class ObOptStatsGatherWholeMsg;
|
||||
// 抽象出本接口类的目的是为了 MsgProc 和 ObPxCoord 解耦
|
||||
class ObIPxCoordMsgProc
|
||||
{
|
||||
@ -46,6 +48,7 @@ public:
|
||||
virtual int on_piece_msg(ObExecContext &ctx, const ObDynamicSamplePieceMsg &pkt) = 0;
|
||||
virtual int on_piece_msg(ObExecContext &ctx, const ObRollupKeyPieceMsg &pkt) = 0;
|
||||
virtual int on_piece_msg(ObExecContext &ctx, const ObRDWFPieceMsg &pkt) = 0;
|
||||
virtual int on_piece_msg(ObExecContext &ctx, const ObOptStatsGatherPieceMsg &pkt) = 0;
|
||||
};
|
||||
|
||||
class ObIPxSubCoordMsgProc
|
||||
@ -69,6 +72,8 @@ public:
|
||||
const ObRollupKeyWholeMsg &pkt) const = 0;
|
||||
virtual int on_whole_msg(
|
||||
const ObRDWFWholeMsg &pkt) const = 0;
|
||||
virtual int on_whole_msg(
|
||||
const ObOptStatsGatherWholeMsg &pkt) const = 0;
|
||||
// SQC 被中断
|
||||
virtual int on_interrupted(const ObInterruptCode &ic) const = 0;
|
||||
};
|
||||
@ -99,6 +104,8 @@ public:
|
||||
const ObRollupKeyWholeMsg &pkt) const;
|
||||
virtual int on_whole_msg(
|
||||
const ObRDWFWholeMsg &pkt) const;
|
||||
virtual int on_whole_msg(
|
||||
const ObOptStatsGatherWholeMsg &pkt) const;
|
||||
private:
|
||||
ObSqcCtx &sqc_ctx_;
|
||||
};
|
||||
|
||||
@ -664,6 +664,7 @@ int ObPxCoordOp::wait_all_running_dfos_exit()
|
||||
ObRollupKeyPieceMsgP rollup_key_piece_msg_proc(ctx_, terminate_msg_proc);
|
||||
ObRDWFPieceMsgP rd_wf_piece_msg_proc(ctx_, terminate_msg_proc);
|
||||
ObPxQcInterruptedP interrupt_proc(ctx_, terminate_msg_proc);
|
||||
ObOptStatsGatherPieceMsgP opt_stats_gather_piece_msg_proc(ctx_, terminate_msg_proc);
|
||||
|
||||
// 这个注册会替换掉旧的proc.
|
||||
(void)msg_loop_.clear_all_proc();
|
||||
@ -676,7 +677,8 @@ int ObPxCoordOp::wait_all_running_dfos_exit()
|
||||
.register_processor(winbuf_piece_msg_proc)
|
||||
.register_processor(sample_piece_msg_proc)
|
||||
.register_processor(rollup_key_piece_msg_proc)
|
||||
.register_processor(rd_wf_piece_msg_proc);
|
||||
.register_processor(rd_wf_piece_msg_proc)
|
||||
.register_processor(opt_stats_gather_piece_msg_proc);
|
||||
loop.ignore_interrupt();
|
||||
|
||||
ObPxControlChannelProc control_channels;
|
||||
@ -735,6 +737,7 @@ int ObPxCoordOp::wait_all_running_dfos_exit()
|
||||
case ObDtlMsgType::DH_DYNAMIC_SAMPLE_PIECE_MSG:
|
||||
case ObDtlMsgType::DH_ROLLUP_KEY_PIECE_MSG:
|
||||
case ObDtlMsgType::DH_RANGE_DIST_WF_PIECE_MSG:
|
||||
case ObDtlMsgType::DH_OPT_STATS_GATHER_PIECE_MSG:
|
||||
break;
|
||||
default:
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
|
||||
@ -14,6 +14,7 @@
|
||||
|
||||
#include "sql/engine/px/ob_px_scheduler.h"
|
||||
#include "sql/engine/px/ob_dfo_scheduler.h"
|
||||
#include "sql/engine/px/ob_dfo_mgr.h"
|
||||
#include "lib/random/ob_random.h"
|
||||
#include "share/ob_rpc_share.h"
|
||||
#include "share/schema/ob_part_mgr_util.h"
|
||||
@ -99,8 +100,6 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
||||
int ObPxMsgProc::on_process_end(ObExecContext &ctx)
|
||||
{
|
||||
UNUSED(ctx);
|
||||
@ -419,6 +418,14 @@ int ObPxMsgProc::on_piece_msg(
|
||||
return proc.on_piece_msg(coord_info_, ctx, pkt);
|
||||
}
|
||||
|
||||
int ObPxMsgProc::on_piece_msg(
|
||||
ObExecContext &ctx,
|
||||
const ObOptStatsGatherPieceMsg &pkt)
|
||||
{
|
||||
ObDhPieceMsgProc<ObOptStatsGatherPieceMsg> proc;
|
||||
return proc.on_piece_msg(coord_info_, ctx, pkt);
|
||||
}
|
||||
|
||||
int ObPxMsgProc::on_eof_row(ObExecContext &ctx)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
@ -794,5 +801,13 @@ int ObPxTerminateMsgProc::on_piece_msg(
|
||||
return common::OB_SUCCESS;
|
||||
}
|
||||
|
||||
int ObPxTerminateMsgProc::on_piece_msg(
|
||||
ObExecContext &,
|
||||
const ObOptStatsGatherPieceMsg &)
|
||||
{
|
||||
return common::OB_SUCCESS;
|
||||
}
|
||||
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
@ -27,6 +27,7 @@
|
||||
#include "sql/engine/px/datahub/components/ob_dh_rollup_key.h"
|
||||
#include "sql/engine/px/datahub/components/ob_dh_barrier.h"
|
||||
#include "sql/engine/px/datahub/components/ob_dh_range_dist_wf.h"
|
||||
#include "sql/engine/px/datahub/components/ob_dh_opt_stats_gather.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
@ -34,6 +35,7 @@ namespace sql
|
||||
{
|
||||
|
||||
class ObPxCoordOp;
|
||||
class ObPxObDfoMgr;
|
||||
class ObPxRootDfoAction
|
||||
{
|
||||
public:
|
||||
@ -121,6 +123,7 @@ public:
|
||||
int on_piece_msg(ObExecContext &ctx, const ObDynamicSamplePieceMsg &pkt);
|
||||
int on_piece_msg(ObExecContext &ctx, const ObRollupKeyPieceMsg &pkt);
|
||||
int on_piece_msg(ObExecContext &ctx, const ObRDWFPieceMsg &pkt);
|
||||
int on_piece_msg(ObExecContext &ctx, const ObOptStatsGatherPieceMsg &pkt);
|
||||
// end DATAHUB msg processing
|
||||
|
||||
ObPxCoordInfo &coord_info_;
|
||||
@ -155,6 +158,7 @@ public:
|
||||
int on_piece_msg(ObExecContext &ctx, const ObDynamicSamplePieceMsg &pkt);
|
||||
int on_piece_msg(ObExecContext &ctx, const ObRollupKeyPieceMsg &pkt);
|
||||
int on_piece_msg(ObExecContext &ctx, const ObRDWFPieceMsg &pkt);
|
||||
int on_piece_msg(ObExecContext &ctx, const ObOptStatsGatherPieceMsg &pkt);
|
||||
// end DATAHUB msg processing
|
||||
private:
|
||||
int do_cleanup_dfo(ObDfo &dfo);
|
||||
@ -170,7 +174,6 @@ private:
|
||||
ObDfoSchedulerBasic *scheduler_;
|
||||
};
|
||||
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
|
||||
@ -99,6 +99,7 @@ int ObPxSQCProxy::setup_loop_proc(ObSqcCtx &sqc_ctx) const
|
||||
.register_processor(sqc_ctx.sample_whole_msg_proc_)
|
||||
.register_processor(sqc_ctx.rollup_key_whole_msg_proc_)
|
||||
.register_processor(sqc_ctx.rd_wf_whole_msg_proc_)
|
||||
.register_processor(sqc_ctx.opt_stats_gather_whole_msg_proc_)
|
||||
.register_interrupt_processor(sqc_ctx.interrupt_proc_);
|
||||
}
|
||||
return ret;
|
||||
|
||||
@ -27,6 +27,7 @@
|
||||
#include "sql/engine/px/datahub/components/ob_dh_rollup_key.h"
|
||||
#include "sql/engine/px/datahub/components/ob_dh_sample.h"
|
||||
#include "sql/engine/px/datahub/components/ob_dh_range_dist_wf.h"
|
||||
#include "sql/engine/px/datahub/components/ob_dh_opt_stats_gather.h"
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace sql
|
||||
@ -50,7 +51,8 @@ public:
|
||||
sqc_proxy_(*this, sqc_arg),
|
||||
all_tasks_finish_(false),
|
||||
interrupted_(false),
|
||||
px_bloom_filter_msg_proc_(msg_proc_)
|
||||
px_bloom_filter_msg_proc_(msg_proc_),
|
||||
opt_stats_gather_whole_msg_proc_(msg_proc_)
|
||||
{}
|
||||
~ObSqcCtx() { reset(); }
|
||||
common::ObIArray<ObPxTask> &get_tasks() { return tasks_; }
|
||||
@ -111,6 +113,7 @@ public:
|
||||
common::ObSEArray<ObPxTabletInfo, 8> partitions_info_;
|
||||
ObPxBloomfilterChProvider bf_ch_provider_;
|
||||
ObPxCreateBloomFilterChannelMsgP px_bloom_filter_msg_proc_;
|
||||
ObOptStatsGatherWholeMsgP opt_stats_gather_whole_msg_proc_;
|
||||
// 用于 datahub 中保存 whole msg provider,一般情况下一个子计划里不会
|
||||
// 超过一个算子会使用 datahub,所以大小默认为 1 即可
|
||||
common::ObSEArray<ObPxDatahubDataProvider *, 1> whole_msg_provider_list_;
|
||||
|
||||
Reference in New Issue
Block a user