diff --git a/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h b/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h index f1874d7234..3423408d4a 100644 --- a/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h +++ b/deps/oblib/src/rpc/obrpc/ob_rpc_packet_list.h @@ -1145,13 +1145,13 @@ PCODE_DEF(OB_TRIM_KEY_LIST, 0x1618) //PCODE_DEF(OB_ALTER_PACKAGE_WITH_RES, 0x161D) //PCODE_DEF(OB_ALTER_TRIGGER_WITH_RES, 0x161E) -//PCODE_DEF(OB_GAIS_BROADCAST_AUTO_INC_CACHE, 0x161F) +PCODE_DEF(OB_GAIS_BROADCAST_AUTO_INC_CACHE, 0x161F) //proxy user //PCODE_DEF(OB_ALTER_USER_PROXY, 0x1620) PCODE_DEF(OB_ALTER_USER_PROXY, 0x1620) // global sequence service -// PCODE_DEF(OB_GAIS_NEXT_SEQUENCE_REQUEST, 0x1621) +PCODE_DEF(OB_GAIS_NEXT_SEQUENCE_REQUEST, 0x1621) // resource limit calculator PCODE_DEF(OB_CAL_UNIT_PHY_RESOURCE, 0x1622) diff --git a/src/diagnose/lua/ob_lua_api.cpp b/src/diagnose/lua/ob_lua_api.cpp index 130f3fcbcb..034c4b6389 100644 --- a/src/diagnose/lua/ob_lua_api.cpp +++ b/src/diagnose/lua/ob_lua_api.cpp @@ -1274,7 +1274,7 @@ int select_sql_workarea_active(lua_State *L) // number_passes gen.next_column(wa_active.profile_.get_number_pass()); // tempseg_size - gen.next_column(wa_active.profile_.get_dumped_size()); + gen.next_column(wa_active.profile_.get_max_dumped_size()); // tenant_id gen.next_column(tenant_id); // policy diff --git a/src/observer/mysql/obmp_query.cpp b/src/observer/mysql/obmp_query.cpp index 8d35bdb0c6..9e1d4594b7 100644 --- a/src/observer/mysql/obmp_query.cpp +++ b/src/observer/mysql/obmp_query.cpp @@ -959,6 +959,8 @@ OB_INLINE int ObMPQuery::do_process(ObSQLSessionInfo &session, //do nothing } else { audit_record.consistency_level_ = plan_ctx->get_consistency_level(); + audit_record.total_memstore_read_row_count_ = plan_ctx->get_total_memstore_read_row_count(); + audit_record.total_ssstore_read_row_count_ = plan_ctx->get_total_ssstore_read_row_count(); } } //update v$sql statistics diff --git a/src/observer/mysql/obmp_stmt_execute.cpp b/src/observer/mysql/obmp_stmt_execute.cpp index 95cd2cc8c3..a0bbba8888 100644 --- a/src/observer/mysql/obmp_stmt_execute.cpp +++ b/src/observer/mysql/obmp_stmt_execute.cpp @@ -1390,6 +1390,8 @@ int ObMPStmtExecute::do_process(ObSQLSessionInfo &session, ObPhysicalPlanCtx *plan_ctx = result.get_exec_context().get_physical_plan_ctx(); if (OB_NOT_NULL(plan_ctx)) { audit_record.consistency_level_ = plan_ctx->get_consistency_level(); + audit_record.total_memstore_read_row_count_ = plan_ctx->get_total_memstore_read_row_count(); + audit_record.total_ssstore_read_row_count_ = plan_ctx->get_total_ssstore_read_row_count(); } } diff --git a/src/observer/ob_srv_xlator_partition.cpp b/src/observer/ob_srv_xlator_partition.cpp index 83e584ade0..9f808790da 100644 --- a/src/observer/ob_srv_xlator_partition.cpp +++ b/src/observer/ob_srv_xlator_partition.cpp @@ -274,6 +274,7 @@ void oceanbase::observer::init_srv_xlator_for_others(ObSrvRpcXlator *xlator) { RPC_PROCESSOR(ObGAISCurrAutoIncP); RPC_PROCESSOR(ObGAISPushAutoIncP); RPC_PROCESSOR(ObGAISClearAutoIncCacheP); + RPC_PROCESSOR(ObGAISNextSequenceP); #ifdef OB_BUILD_SPM // sql plan baseline @@ -325,4 +326,6 @@ void oceanbase::observer::init_srv_xlator_for_others(ObSrvRpcXlator *xlator) { // ddl RPC_PROCESSOR(ObRpcCheckandCancelDDLComplementDagP, gctx_); + + RPC_PROCESSOR(ObGAISBroadcastAutoIncCacheP); } diff --git a/src/observer/table/ob_table_context.cpp b/src/observer/table/ob_table_context.cpp index eed32230b4..85d599ea9b 100644 --- a/src/observer/table/ob_table_context.cpp +++ b/src/observer/table/ob_table_context.cpp @@ -1693,7 +1693,8 @@ int ObTableCtx::add_auto_inc_param(const ObColumnSchemaV2 &column_schema) param.autoinc_first_part_num_ = table_schema_->get_first_part_num(); param.autoinc_table_part_num_ = table_schema_->get_all_part_num(); param.autoinc_col_id_ = column_schema.get_column_id(); - param.auto_increment_cache_size_ = auto_increment_cache_size; + param.auto_increment_cache_size_ = get_auto_increment_cache_size( + table_schema_->get_auto_increment_cache_size(), auto_increment_cache_size); param.part_level_ = table_schema_->get_part_level(); ObObjType column_type = column_schema.get_data_type(); param.autoinc_col_type_ = column_type; diff --git a/src/observer/virtual_table/ob_all_virtual_dtl_interm_result_monitor.cpp b/src/observer/virtual_table/ob_all_virtual_dtl_interm_result_monitor.cpp index c53a81397f..bd8eb75181 100644 --- a/src/observer/virtual_table/ob_all_virtual_dtl_interm_result_monitor.cpp +++ b/src/observer/virtual_table/ob_all_virtual_dtl_interm_result_monitor.cpp @@ -124,7 +124,7 @@ int ObDTLIntermResultMonitorInfoGetter::operator() (common::hash::HashMapPair 0) { + // qc thread + cells[cell_idx].set_int(record.data_.exec_record_.memstore_read_row_count_ + + record.data_.total_memstore_read_row_count_); + } else { + // work thread + cells[cell_idx].set_int(record.data_.exec_record_.memstore_read_row_count_); + } } break; case TOTAL_SSSTORE_READ_ROW_COUNT: { - cells[cell_idx].set_int(record.data_.total_ssstore_read_row_count_); + if (record.data_.sql_len_ > 0) { + // qc thread + cells[cell_idx].set_int(record.data_.exec_record_.ssstore_read_row_count_ + + record.data_.total_ssstore_read_row_count_); + } else { + // work thread + cells[cell_idx].set_int(record.data_.exec_record_.ssstore_read_row_count_); + } } break; case PROXY_USER_NAME: { int64_t len = min(record.data_.proxy_user_name_len_, OB_MAX_USER_NAME_LENGTH); diff --git a/src/observer/virtual_table/ob_virtual_sql_plan_monitor.cpp b/src/observer/virtual_table/ob_virtual_sql_plan_monitor.cpp index a7b0d8ff0b..d1bdadc82f 100644 --- a/src/observer/virtual_table/ob_virtual_sql_plan_monitor.cpp +++ b/src/observer/virtual_table/ob_virtual_sql_plan_monitor.cpp @@ -829,19 +829,31 @@ int ObVirtualSqlPlanMonitor::convert_node_to_row(ObMonitorNode &node, ObNewRow * break; } case WORKAREA_MEM: { - cells[cell_idx].set_null(); + if(need_rt_node_) { + int64_t int_value = node.workarea_mem_; + cells[cell_idx].set_int(int_value); + } else { + cells[cell_idx].set_null(); + } break; } case WORKAREA_MAX_MEM: { - cells[cell_idx].set_null(); + int64_t int_value = node.workarea_max_mem_; + cells[cell_idx].set_int(int_value); break; } case WORKAREA_TEMPSEG: { - cells[cell_idx].set_null(); + if (need_rt_node_) { + int64_t int_value = node.workarea_tempseg_; + cells[cell_idx].set_int(int_value); + } else { + cells[cell_idx].set_null(); + } break; } case WORKAREA_MAX_TEMPSEG: { - cells[cell_idx].set_null(); + int64_t int_value = node.workarea_max_tempseg_; + cells[cell_idx].set_int(int_value); break; } default: { diff --git a/src/rootserver/ddl_task/ob_ddl_redefinition_task.cpp b/src/rootserver/ddl_task/ob_ddl_redefinition_task.cpp index 07ed4ad950..b626ff2581 100644 --- a/src/rootserver/ddl_task/ob_ddl_redefinition_task.cpp +++ b/src/rootserver/ddl_task/ob_ddl_redefinition_task.cpp @@ -1181,7 +1181,7 @@ int ObDDLRedefinitionTask::sync_auto_increment_position() param.autoinc_desired_count_ = 0; param.autoinc_increment_ = 1; param.autoinc_offset_ = 1; - param.auto_increment_cache_size_ = 1; // TODO(shuangcan): should we use the sysvar on session? + param.auto_increment_cache_size_ = 0; // set cache size to 0 to disable prefetch param.autoinc_mode_is_order_ = dest_table_schema->is_order_auto_increment_mode(); param.autoinc_auto_increment_ = dest_table_schema->get_auto_increment(); param.autoinc_version_ = dest_table_schema->get_truncate_version(); @@ -1326,7 +1326,8 @@ int ObDDLRedefinitionTask::modify_autoinc(const ObDDLTaskStatus next_task_status param.autoinc_increment_ = 1; param.autoinc_offset_ = 1; param.global_value_to_sync_ = autoinc_val - 1; - param.auto_increment_cache_size_ = 1; // TODO(shuangcan): should we use the sysvar on session? + param.auto_increment_cache_size_ = 0; // set cache size to 0 to disable prefetch + param.autoinc_mode_is_order_ = new_table_schema->is_order_auto_increment_mode(); param.autoinc_version_ = new_table_schema->get_truncate_version(); if (OB_FAIL(auto_inc_service.sync_insert_value_global(param))) { LOG_WARN("fail to clear autoinc cache", K(ret), K(param)); diff --git a/src/rootserver/ob_ddl_service.cpp b/src/rootserver/ob_ddl_service.cpp index 644a83ad50..02138a6733 100755 --- a/src/rootserver/ob_ddl_service.cpp +++ b/src/rootserver/ob_ddl_service.cpp @@ -3349,6 +3349,22 @@ int ObDDLService::set_raw_table_options( alter_table_schema.get_table_auto_increment_mode()); break; } + case ObAlterTableArg::INCREMENT_CACHE_SIZE : { + uint64_t data_version = OB_INVALID_VERSION; + if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, data_version))) { + LOG_WARN("get min data_version failed", K(ret), K(tenant_id)); + } else if (data_version < MOCK_DATA_VERSION_4_2_3_0 || + (data_version >= DATA_VERSION_4_3_0_0 && data_version < DATA_VERSION_4_3_2_0)) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("table auto_increment_cache_size less than 4.2.3 not support", K(ret), + K(data_version)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, "table auto_increment_cache_size less than 4.2.3"); + } else { + new_table_schema.set_auto_increment_cache_size( + alter_table_schema.get_auto_increment_cache_size()); + } + break; + } case ObAlterTableArg::ENABLE_EXTENDED_ROWID: { new_table_schema.set_table_rowid_mode(alter_table_schema.get_table_rowid_mode()); break; @@ -10812,6 +10828,117 @@ int ObDDLService::alter_not_null_cst_in_column_flag( return ret; } +int ObDDLService::alter_table_auto_increment( + const ObTableSchema &orig_table_schema, + const AlterTableSchema &alter_table_schema, + const obrpc::ObAlterTableArg &alter_table_arg, + share::schema::ObSchemaGetterGuard &schema_guard, + ObTableSchema &new_table_schema, + ObDDLOperator &ddl_operator, + ObMySQLTransaction &trans) +{ + int ret = OB_SUCCESS; + ObAutoincrementService &auto_inc_service = ObAutoincrementService::get_instance(); + const uint64_t tenant_id = orig_table_schema.get_tenant_id(); + const uint64_t table_id = orig_table_schema.get_table_id(); + const uint64_t column_id = orig_table_schema.get_autoinc_column_id(); + const bool is_order_mode = orig_table_schema.is_order_auto_increment_mode(); + const int64_t truncate_version = orig_table_schema.get_truncate_version(); + uint64_t current_auto_increment = 0; + // Step 1: Determine whether the auto-increment value needs to be increased or decreased. + // If increased, only the table schema needs to be updated. It reduces the complexity of the + // alter_table operation. + const bool is_reduced_autoinc = + (alter_table_schema.get_auto_increment() < orig_table_schema.get_auto_increment()); + if (!is_reduced_autoinc && OB_FAIL(auto_inc_service.get_sequence_value(tenant_id, + table_id, column_id, is_order_mode, truncate_version, current_auto_increment))) { + LOG_WARN("fail to get sequence value", K(ret)); + } else if (is_reduced_autoinc || + (alter_table_schema.get_auto_increment() < current_auto_increment)) { + // Step 2: Query the maximum value of the auto-increment column and use it as a base value. + const ObDatabaseSchema *db_schema = nullptr; + const ObColumnSchemaV2 *column_schema = nullptr; + if (OB_FAIL(schema_guard.get_database_schema(tenant_id, orig_table_schema.get_database_id(), + db_schema))) { + LOG_WARN("failed to get database schema", K(ret), K(orig_table_schema.get_data_table_id())); + } else if (OB_ISNULL(db_schema)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("error unexpected, database schema must not be nullptr", K(ret)); + } else if (OB_ISNULL(column_schema = orig_table_schema.get_column_schema(column_id))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("failed to get alter column schema", K(ret), K(column_id)); + } else if (OB_FAIL(lock_table(trans, orig_table_schema))) { + LOG_WARN("failed to lock table for alter auto_increment", K(ret)); + } else { + uint64_t current_max_value = 0; + SMART_VAR(ObMySQLProxy::MySQLResult, res) { + ObTimeoutCtx timeout_ctx; + ObSqlString sql; + sqlclient::ObMySQLResult *result = NULL; + common::ObCommonSqlProxy *user_sql_proxy = GCTX.ddl_sql_proxy_; + ObSessionParam session_param; + int64_t sql_mode = alter_table_arg.sql_mode_; + session_param.sql_mode_ = reinterpret_cast(&sql_mode); + session_param.ddl_info_.set_is_ddl(true); + // if data_table_id != dest_table_id, meaning this is happening in ddl double write + session_param.ddl_info_.set_source_table_hidden(orig_table_schema.is_user_hidden_table()); + ObObj obj; + const int64_t DDL_INNER_SQL_EXECUTE_TIMEOUT = ObDDLUtil::calc_inner_sql_execute_timeout(); + const bool is_unsigned_type = ob_is_unsigned_type(column_schema->get_data_type()); + if (OB_FAIL(timeout_ctx.set_trx_timeout_us(DDL_INNER_SQL_EXECUTE_TIMEOUT))) { + LOG_WARN("set trx timeout failed", K(ret)); + } else if (OB_FAIL(timeout_ctx.set_timeout(DDL_INNER_SQL_EXECUTE_TIMEOUT))) { + LOG_WARN("set timeout failed", K(ret)); + } else if (OB_FAIL(sql.assign_fmt("SELECT /*+no_rewrite*/ CAST(MAX(`%s`) AS %s) AS MAX_VALUE FROM `%s`.`%s`", + column_schema->get_column_name(), + is_unsigned_type ? "UNSIGNED" : "SIGNED", + db_schema->get_database_name(), + orig_table_schema.get_table_name()))) { + LOG_WARN("failed to assign fmt", K(ret), K(column_schema->get_column_name_str()), + K(db_schema->get_database_name_str())); + } else if (OB_FAIL(user_sql_proxy->read(res, tenant_id, sql.ptr(), &session_param))) { + LOG_WARN("fail to read", KR(ret), K(sql)); + } else if (OB_ISNULL(result = res.get_result())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get result failed", K(ret)); + } else if (OB_FAIL(result->next())) { + if (OB_ITER_END == ret) { + // empty table + ret = OB_SUCCESS; + } else { + LOG_WARN("fail to get next", KR(ret)); + } + } else if (OB_FAIL(result->get_obj("MAX_VALUE", obj))) { + LOG_WARN("fail to get result obj", K(ret)); + } else if (is_unsigned_type) { + current_max_value = obj.get_uint64(); + } else { + current_max_value = MAX(0, obj.get_int()); + } + } + + // Step 3: Clear the auto increment cache and reset the inner table of the auto-increment + // column. all new auto_inc request will be based on the auto_increment value in the table + // schema. + if (OB_FAIL(ret)) { + } else if (OB_FAIL(auto_inc_service.clear_autoinc_cache_all(tenant_id, table_id, + column_id, is_order_mode, + false /* ignore_rpc_errors */))) { + LOG_WARN("fail to clear autoinc cache all", K(ret)); + } else if (OB_FAIL(ddl_operator.reinit_autoinc_row(new_table_schema, trans))) { + LOG_WARN("fail to reinit autoinc row", K(ret)); + } else { + // Step 4: Update the table schema with the maximum values of auto_increment by setting and + // max_value plus 1. + const uint64_t next_value = + (current_max_value == UINT64_MAX) ? UINT64_MAX : (current_max_value + 1); + new_table_schema.set_auto_increment(MAX(next_value, new_table_schema.get_auto_increment())); + } + } + } + return ret; +} + int ObDDLService::alter_table_constraints(const ObAlterTableArg::AlterConstraintType op_type, share::schema::ObSchemaGetterGuard &schema_guard, const ObTableSchema &orig_table_schema, @@ -12573,6 +12700,21 @@ int ObDDLService::alter_table_in_trans(obrpc::ObAlterTableArg &alter_table_arg, } } + // alter table auto increment value + if (OB_SUCC(ret) && + alter_table_schema.alter_option_bitset_.has_member(ObAlterTableArg::AUTO_INCREMENT) && + 0 != new_table_schema.get_autoinc_column_id()) { + if (OB_FAIL(alter_table_auto_increment(*orig_table_schema, + alter_table_schema, + alter_table_arg, + schema_guard, + new_table_schema, + ddl_operator, + trans))) { + LOG_WARN("fail to alter table auto increment value", K(ret)); + } + } + if (OB_SUCC(ret)) { ObSchemaOperationType operation_type = OB_DDL_ALTER_TABLE; if (obrpc::ObAlterTableArg::PARTITIONED_TABLE == alter_table_arg.alter_part_type_) { diff --git a/src/rootserver/ob_ddl_service.h b/src/rootserver/ob_ddl_service.h index e5e5c54812..35b0d2ac99 100644 --- a/src/rootserver/ob_ddl_service.h +++ b/src/rootserver/ob_ddl_service.h @@ -515,6 +515,15 @@ public: ObTableSchema &new_table_schema, ObDDLOperator &ddl_operator, ObMySQLTransaction &trans); + + int alter_table_auto_increment(const ObTableSchema &orig_table_schema, + const AlterTableSchema &alter_table_schema, + const obrpc::ObAlterTableArg &alter_table_arg, + share::schema::ObSchemaGetterGuard &schema_guard, + ObTableSchema &new_table_schema, + ObDDLOperator &ddl_operator, + ObMySQLTransaction &trans); + int get_tablets( const uint64_t tenant_id, const ObArray &tablet_ids, diff --git a/src/share/diagnosis/ob_sql_plan_monitor_node_list.cpp b/src/share/diagnosis/ob_sql_plan_monitor_node_list.cpp index 825e7d6f99..27d9363f56 100644 --- a/src/share/diagnosis/ob_sql_plan_monitor_node_list.cpp +++ b/src/share/diagnosis/ob_sql_plan_monitor_node_list.cpp @@ -201,6 +201,18 @@ int ObPlanMonitorNodeList::ObMonitorNodeTraverseCall::operator() ( return ret; } +void ObMonitorNode::update_memory(int64_t delta_size) +{ + workarea_mem_ += delta_size; + workarea_max_mem_ = MAX(workarea_mem_, workarea_max_mem_); +} + +void ObMonitorNode::update_tempseg(int64_t delta_size) +{ + workarea_tempseg_ += delta_size; + workarea_max_tempseg_ = MAX(workarea_tempseg_, workarea_max_tempseg_); +} + void ObSqlPlanMonitorRecycleTask::runTimerTask() { if (node_list_) { diff --git a/src/share/diagnosis/ob_sql_plan_monitor_node_list.h b/src/share/diagnosis/ob_sql_plan_monitor_node_list.h index e981f64964..546abe09e1 100644 --- a/src/share/diagnosis/ob_sql_plan_monitor_node_list.h +++ b/src/share/diagnosis/ob_sql_plan_monitor_node_list.h @@ -68,7 +68,6 @@ public: output_row_count_(0), db_time_(0), block_time_(0), - memory_used_(0), disk_read_count_(0), otherstat_1_value_(0), otherstat_2_value_(0), @@ -82,7 +81,11 @@ public: otherstat_4_id_(0), otherstat_5_id_(0), otherstat_6_id_(0), - enable_rich_format_(false) + enable_rich_format_(false), + workarea_mem_(0), + workarea_max_mem_(0), + workarea_tempseg_(0), + workarea_max_tempseg_(0) { TraceId* trace_id = common::ObCurTraceId::get_trace_id(); if (NULL != trace_id) { @@ -111,6 +114,8 @@ public: int64_t get_rt_node_id() { return rt_node_id_;} int add_rt_monitor_node(ObMonitorNode *node); void set_rich_format(bool v) { enable_rich_format_ = v; } + void update_memory(int64_t delta_size); + void update_tempseg(int64_t delta_size); TO_STRING_KV(K_(tenant_id), K_(op_id), "op_name", get_operator_name(), K_(thread_id)); public: int64_t tenant_id_; @@ -133,7 +138,6 @@ public: int64_t output_row_count_; uint64_t db_time_; // rdtsc cpu cycles spend on this op, include cpu instructions & io uint64_t block_time_; // rdtsc cpu cycles wait for network, io etc - int64_t memory_used_; int64_t disk_read_count_; // 各个算子特有的信息 int64_t otherstat_1_value_; @@ -149,6 +153,10 @@ public: int16_t otherstat_5_id_; int16_t otherstat_6_id_; bool enable_rich_format_; + int64_t workarea_mem_; + int64_t workarea_max_mem_; + int64_t workarea_tempseg_; + int64_t workarea_max_tempseg_; }; diff --git a/src/share/inner_table/ob_inner_table_schema.21001_21050.cpp b/src/share/inner_table/ob_inner_table_schema.21001_21050.cpp index c77f3a6135..946ca01644 100644 --- a/src/share/inner_table/ob_inner_table_schema.21001_21050.cpp +++ b/src/share/inner_table/ob_inner_table_schema.21001_21050.cpp @@ -1660,7 +1660,7 @@ int ObInnerTableSchema::gv_sql_plan_monitor_schema(ObTableSchema &table_schema) table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); if (OB_SUCC(ret)) { - if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT TENANT_ID as CON_ID, REQUEST_ID, CAST(NULL as UNSIGNED) AS `KEY`, CAST(NULL AS CHAR(19)) as STATUS, SVR_IP, SVR_PORT, TRACE_ID, DB_TIME, USER_IO_WAIT_TIME, CAST(NULL AS UNSIGNED) AS OTHER_WAIT_TIME, FIRST_REFRESH_TIME, LAST_REFRESH_TIME, FIRST_CHANGE_TIME, LAST_CHANGE_TIME, CAST(NULL AS UNSIGNED) AS REFRESH_COUNT, CAST(NULL AS UNSIGNED) AS SID, THREAD_ID PROCESS_NAME, CAST(NULL AS CHAR(32)) AS SQL_ID, CAST(NULL AS UNSIGNED) AS SQL_EXEC_START, CAST(NULL AS UNSIGNED) AS SQL_EXEC_ID, CAST(NULL AS UNSIGNED) AS SQL_PLAN_HASH_VALUE, CAST(NULL AS BINARY(8)) AS SQL_CHILD_ADDRESS, CAST(NULL AS UNSIGNED) AS PLAN_PARENT_ID, PLAN_LINE_ID, PLAN_OPERATION, CAST(NULL AS CHAR(30)) PLAN_OPTIONS, CAST(NULL AS CHAR(128)) PLAN_OBJECT_OWNER, CAST(NULL AS CHAR(128)) PLAN_OBJECT_NAME, CAST(NULL AS CHAR(80)) PLAN_OBJECT_TYPE, PLAN_DEPTH, CAST( NULL AS UNSIGNED) AS PLAN_POSITION, CAST( NULL AS UNSIGNED) AS PLAN_COST, CAST( NULL AS UNSIGNED) AS PLAN_CARDINALITY, CAST( NULL AS UNSIGNED) AS PLAN_BYTES, CAST( NULL AS UNSIGNED) AS PLAN_TIME, CAST( NULL AS UNSIGNED) AS PLAN_PARTITION_START, CAST( NULL AS UNSIGNED) AS PLAN_PARTITION_STOP, CAST( NULL AS UNSIGNED) AS PLAN_CPU_COST, CAST( NULL AS UNSIGNED) AS PLAN_IO_COST, CAST( NULL AS UNSIGNED) AS PLAN_TEMP_SPACE, STARTS, OUTPUT_ROWS, CAST( NULL AS UNSIGNED) AS IO_INTERCONNECT_BYTES, CAST( NULL AS UNSIGNED) AS PHYSICAL_READ_REQUESTS, CAST( NULL AS UNSIGNED) AS PHYSICAL_READ_BYTES, CAST( NULL AS UNSIGNED) AS PHYSICAL_WRITE_REQUESTS, CAST( NULL AS UNSIGNED) AS PHYSICAL_WRITE_BYTES, CAST( NULL AS UNSIGNED) AS WORKAREA_MEM, CAST( NULL AS UNSIGNED) AS WORKAREA_MAX_MEM, CAST( NULL AS UNSIGNED) AS WORKAREA_TEMPSEG, CAST( NULL AS UNSIGNED) AS WORKAREA_MAX_TEMPSEG, CAST( NULL AS UNSIGNED) AS OTHERSTAT_GROUP_ID, OTHERSTAT_1_ID, CAST(NULL AS UNSIGNED) AS OTHERSTAT_1_TYPE, OTHERSTAT_1_VALUE, OTHERSTAT_2_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_2_TYPE, OTHERSTAT_2_VALUE, OTHERSTAT_3_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_3_TYPE, OTHERSTAT_3_VALUE, OTHERSTAT_4_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_4_TYPE, OTHERSTAT_4_VALUE, OTHERSTAT_5_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_5_TYPE, OTHERSTAT_5_VALUE, OTHERSTAT_6_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_6_TYPE, OTHERSTAT_6_VALUE, OTHERSTAT_7_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_7_TYPE, OTHERSTAT_7_VALUE, OTHERSTAT_8_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_8_TYPE, OTHERSTAT_8_VALUE, OTHERSTAT_9_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_9_TYPE, OTHERSTAT_9_VALUE, OTHERSTAT_10_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_10_TYPE, OTHERSTAT_10_VALUE, CAST(NULL AS CHAR(255)) AS OTHER_XML, CAST(NULL AS UNSIGNED) AS PLAN_OPERATION_INACTIVE, OUTPUT_BATCHES, SKIPPED_ROWS_COUNT FROM oceanbase.__all_virtual_sql_plan_monitor )__"))) { + if (OB_FAIL(table_schema.set_view_definition(R"__( SELECT TENANT_ID as CON_ID, REQUEST_ID, CAST(NULL as UNSIGNED) AS `KEY`, CAST(NULL AS CHAR(19)) as STATUS, SVR_IP, SVR_PORT, TRACE_ID, DB_TIME, USER_IO_WAIT_TIME, CAST(NULL AS UNSIGNED) AS OTHER_WAIT_TIME, FIRST_REFRESH_TIME, LAST_REFRESH_TIME, FIRST_CHANGE_TIME, LAST_CHANGE_TIME, CAST(NULL AS UNSIGNED) AS REFRESH_COUNT, CAST(NULL AS UNSIGNED) AS SID, THREAD_ID PROCESS_NAME, CAST(NULL AS CHAR(32)) AS SQL_ID, CAST(NULL AS UNSIGNED) AS SQL_EXEC_START, CAST(NULL AS UNSIGNED) AS SQL_EXEC_ID, CAST(NULL AS UNSIGNED) AS SQL_PLAN_HASH_VALUE, CAST(NULL AS BINARY(8)) AS SQL_CHILD_ADDRESS, CAST(NULL AS UNSIGNED) AS PLAN_PARENT_ID, PLAN_LINE_ID, PLAN_OPERATION, CAST(NULL AS CHAR(30)) PLAN_OPTIONS, CAST(NULL AS CHAR(128)) PLAN_OBJECT_OWNER, CAST(NULL AS CHAR(128)) PLAN_OBJECT_NAME, CAST(NULL AS CHAR(80)) PLAN_OBJECT_TYPE, PLAN_DEPTH, CAST( NULL AS UNSIGNED) AS PLAN_POSITION, CAST( NULL AS UNSIGNED) AS PLAN_COST, CAST( NULL AS UNSIGNED) AS PLAN_CARDINALITY, CAST( NULL AS UNSIGNED) AS PLAN_BYTES, CAST( NULL AS UNSIGNED) AS PLAN_TIME, CAST( NULL AS UNSIGNED) AS PLAN_PARTITION_START, CAST( NULL AS UNSIGNED) AS PLAN_PARTITION_STOP, CAST( NULL AS UNSIGNED) AS PLAN_CPU_COST, CAST( NULL AS UNSIGNED) AS PLAN_IO_COST, CAST( NULL AS UNSIGNED) AS PLAN_TEMP_SPACE, STARTS, OUTPUT_ROWS, CAST( NULL AS UNSIGNED) AS IO_INTERCONNECT_BYTES, CAST( NULL AS UNSIGNED) AS PHYSICAL_READ_REQUESTS, CAST( NULL AS UNSIGNED) AS PHYSICAL_READ_BYTES, CAST( NULL AS UNSIGNED) AS PHYSICAL_WRITE_REQUESTS, CAST( NULL AS UNSIGNED) AS PHYSICAL_WRITE_BYTES, CAST( WORKAREA_MEM AS UNSIGNED) AS WORKAREA_MEM, CAST( WORKAREA_MAX_MEM AS UNSIGNED) AS WORKAREA_MAX_MEM, CAST( WORKAREA_TEMPSEG AS UNSIGNED) AS WORKAREA_TEMPSEG, CAST( WORKAREA_MAX_TEMPSEG AS UNSIGNED) AS WORKAREA_MAX_TEMPSEG, CAST( NULL AS UNSIGNED) AS OTHERSTAT_GROUP_ID, OTHERSTAT_1_ID, CAST(NULL AS UNSIGNED) AS OTHERSTAT_1_TYPE, OTHERSTAT_1_VALUE, OTHERSTAT_2_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_2_TYPE, OTHERSTAT_2_VALUE, OTHERSTAT_3_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_3_TYPE, OTHERSTAT_3_VALUE, OTHERSTAT_4_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_4_TYPE, OTHERSTAT_4_VALUE, OTHERSTAT_5_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_5_TYPE, OTHERSTAT_5_VALUE, OTHERSTAT_6_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_6_TYPE, OTHERSTAT_6_VALUE, OTHERSTAT_7_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_7_TYPE, OTHERSTAT_7_VALUE, OTHERSTAT_8_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_8_TYPE, OTHERSTAT_8_VALUE, OTHERSTAT_9_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_9_TYPE, OTHERSTAT_9_VALUE, OTHERSTAT_10_ID, CAST(NULL AS UNSIGNED) OTHERSTAT_10_TYPE, OTHERSTAT_10_VALUE, CAST(NULL AS CHAR(255)) AS OTHER_XML, CAST(NULL AS UNSIGNED) AS PLAN_OPERATION_INACTIVE, OUTPUT_BATCHES, SKIPPED_ROWS_COUNT FROM oceanbase.__all_virtual_sql_plan_monitor )__"))) { LOG_ERROR("fail to set view_definition", K(ret)); } } diff --git a/src/share/inner_table/ob_inner_table_schema.28051_28100.cpp b/src/share/inner_table/ob_inner_table_schema.28051_28100.cpp index ebb30f004e..286e554921 100644 --- a/src/share/inner_table/ob_inner_table_schema.28051_28100.cpp +++ b/src/share/inner_table/ob_inner_table_schema.28051_28100.cpp @@ -510,7 +510,7 @@ int ObInnerTableSchema::gv_sql_plan_monitor_ora_schema(ObTableSchema &table_sche table_schema.set_collation_type(ObCharset::get_default_collation(ObCharset::get_default_charset())); if (OB_SUCC(ret)) { - if (OB_FAIL(table_schema.set_view_definition(R"__(SELECT CAST(TENANT_ID AS NUMBER) CON_ID, REQUEST_ID, CAST(NULL AS NUMBER) KEY, CAST(NULL AS VARCHAR2(19)) STATUS, SVR_IP, SVR_PORT, TRACE_ID, CAST(FIRST_REFRESH_TIME AS TIMESTAMP) FIRST_REFRESH_TIME, CAST(LAST_REFRESH_TIME AS TIMESTAMP) LAST_REFRESH_TIME, CAST(FIRST_CHANGE_TIME AS TIMESTAMP) FIRST_CHANGE_TIME, CAST(LAST_CHANGE_TIME AS TIMESTAMP) LAST_CHANGE_TIME, CAST(NULL AS NUMBER) REFRESH_COUNT, CAST(NULL AS NUMBER) SID, CAST(THREAD_ID AS VARCHAR2(10)) PROCESS_NAME, CAST(NULL AS VARCHAR2(13)) SQL_ID, CAST(NULL AS TIMESTAMP) SQL_EXEC_START, CAST(NULL AS NUMBER) SQL_EXEC_ID, CAST(NULL AS NUMBER) SQL_PLAN_HASH_VALUE, CAST(NULL AS RAW(8)) SQL_CHILD_ADDRESS, CAST(NULL AS NUMBER) PLAN_PARENT_ID, CAST(PLAN_LINE_ID AS NUMBER) PLAN_LINE_ID, CAST(PLAN_OPERATION AS VARCHAR2(30)) PLAN_OPERATION, CAST(NULL AS VARCHAR2(30)) PLAN_OPTIONS, CAST(NULL AS VARCHAR2(128)) PLAN_OBJECT_OWNER, CAST(NULL AS VARCHAR2(128)) PLAN_OBJECT_NAME, CAST(NULL AS VARCHAR2(80)) PLAN_OBJECT_TYPE, CAST(PLAN_DEPTH AS NUMBER) PLAN_DEPTH, CAST(NULL AS NUMBER) PLAN_POSITION, CAST(NULL AS NUMBER) PLAN_COST, CAST(NULL AS NUMBER) PLAN_CARDINALITY, CAST(NULL AS NUMBER) PLAN_BYTES, CAST(NULL AS NUMBER) PLAN_TIME, CAST(NULL AS VARCHAR2(256)) PLAN_PARTITION_START, CAST(NULL AS VARCHAR2(256)) PLAN_PARTITION_STOP, CAST(NULL AS NUMBER) PLAN_CPU_COST, CAST(NULL AS NUMBER) PLAN_IO_COST, CAST(NULL AS NUMBER) PLAN_TEMP_SPACE, CAST(STARTS AS NUMBER) STARTS, CAST(OUTPUT_ROWS AS NUMBER) OUTPUT_ROWS, CAST(NULL AS NUMBER) IO_INTERCONNECT_BYTES, CAST(NULL AS NUMBER) PHYSICAL_READ_REQUESTS, CAST(NULL AS NUMBER) PHYSICAL_READ_BYTES, CAST(NULL AS NUMBER) PHYSICAL_WRITE_REQUESTS, CAST(NULL AS NUMBER) PHYSICAL_WRITE_BYTES, CAST(NULL AS NUMBER) WORKAREA_MEM, CAST(NULL AS NUMBER) WORKAREA_MAX_MEM, CAST(NULL AS NUMBER) WORKAREA_TEMPSEG, CAST(NULL AS NUMBER) WORKAREA_MAX_TEMPSEG, CAST(NULL AS NUMBER) OTHERSTAT_GROUP_ID, CAST(OTHERSTAT_1_ID AS NUMBER) OTHERSTAT_1_ID, CAST(NULL AS NUMBER) OTHERSTAT_1_TYPE, CAST(OTHERSTAT_1_VALUE AS NUMBER) OTHERSTAT_1_VALUE, CAST(OTHERSTAT_2_ID AS NUMBER) OTHERSTAT_2_ID, CAST(NULL AS NUMBER) OTHERSTAT_2_TYPE, CAST(OTHERSTAT_2_VALUE AS NUMBER) OTHERSTAT_2_VALUE, CAST(OTHERSTAT_3_ID AS NUMBER) OTHERSTAT_3_ID, CAST(NULL AS NUMBER) OTHERSTAT_3_TYPE, CAST(OTHERSTAT_3_VALUE AS NUMBER) OTHERSTAT_3_VALUE, CAST(OTHERSTAT_4_ID AS NUMBER) OTHERSTAT_4_ID, CAST(NULL AS NUMBER) OTHERSTAT_4_TYPE, CAST(OTHERSTAT_4_VALUE AS NUMBER) OTHERSTAT_4_VALUE, CAST(OTHERSTAT_5_ID AS NUMBER) OTHERSTAT_5_ID, CAST(NULL AS NUMBER) OTHERSTAT_5_TYPE, CAST(OTHERSTAT_5_VALUE AS NUMBER) OTHERSTAT_5_VALUE, CAST(OTHERSTAT_6_ID AS NUMBER) OTHERSTAT_6_ID, CAST(NULL AS NUMBER) OTHERSTAT_6_TYPE, CAST(OTHERSTAT_6_VALUE AS NUMBER) OTHERSTAT_6_VALUE, CAST(OTHERSTAT_7_ID AS NUMBER) OTHERSTAT_7_ID, CAST(NULL AS NUMBER) OTHERSTAT_7_TYPE, CAST(OTHERSTAT_7_VALUE AS NUMBER) OTHERSTAT_7_VALUE, CAST(OTHERSTAT_8_ID AS NUMBER) OTHERSTAT_8_ID, CAST(NULL AS NUMBER) OTHERSTAT_8_TYPE, CAST(OTHERSTAT_8_VALUE AS NUMBER) OTHERSTAT_8_VALUE, CAST(OTHERSTAT_9_ID AS NUMBER) OTHERSTAT_9_ID, CAST(NULL AS NUMBER) OTHERSTAT_9_TYPE, CAST(OTHERSTAT_9_VALUE AS NUMBER) OTHERSTAT_9_VALUE, CAST(OTHERSTAT_10_ID AS NUMBER) OTHERSTAT_10_ID, CAST(NULL AS NUMBER) OTHERSTAT_10_TYPE, CAST(OTHERSTAT_10_VALUE AS NUMBER) OTHERSTAT_10_VALUE, CAST(NULL AS VARCHAR(1)) OTHER_XML, CAST(NULL AS NUMBER) PLAN_OPERATION_INACTIVE, OUTPUT_BATCHES, SKIPPED_ROWS_COUNT, DB_TIME, USER_IO_WAIT_TIME, NULL OTHER_WAIT_TIME FROM SYS.ALL_VIRTUAL_SQL_PLAN_MONITOR )__"))) { + if (OB_FAIL(table_schema.set_view_definition(R"__(SELECT CAST(TENANT_ID AS NUMBER) CON_ID, REQUEST_ID, CAST(NULL AS NUMBER) KEY, CAST(NULL AS VARCHAR2(19)) STATUS, SVR_IP, SVR_PORT, TRACE_ID, CAST(FIRST_REFRESH_TIME AS TIMESTAMP) FIRST_REFRESH_TIME, CAST(LAST_REFRESH_TIME AS TIMESTAMP) LAST_REFRESH_TIME, CAST(FIRST_CHANGE_TIME AS TIMESTAMP) FIRST_CHANGE_TIME, CAST(LAST_CHANGE_TIME AS TIMESTAMP) LAST_CHANGE_TIME, CAST(NULL AS NUMBER) REFRESH_COUNT, CAST(NULL AS NUMBER) SID, CAST(THREAD_ID AS VARCHAR2(10)) PROCESS_NAME, CAST(NULL AS VARCHAR2(13)) SQL_ID, CAST(NULL AS TIMESTAMP) SQL_EXEC_START, CAST(NULL AS NUMBER) SQL_EXEC_ID, CAST(NULL AS NUMBER) SQL_PLAN_HASH_VALUE, CAST(NULL AS RAW(8)) SQL_CHILD_ADDRESS, CAST(NULL AS NUMBER) PLAN_PARENT_ID, CAST(PLAN_LINE_ID AS NUMBER) PLAN_LINE_ID, CAST(PLAN_OPERATION AS VARCHAR2(30)) PLAN_OPERATION, CAST(NULL AS VARCHAR2(30)) PLAN_OPTIONS, CAST(NULL AS VARCHAR2(128)) PLAN_OBJECT_OWNER, CAST(NULL AS VARCHAR2(128)) PLAN_OBJECT_NAME, CAST(NULL AS VARCHAR2(80)) PLAN_OBJECT_TYPE, CAST(PLAN_DEPTH AS NUMBER) PLAN_DEPTH, CAST(NULL AS NUMBER) PLAN_POSITION, CAST(NULL AS NUMBER) PLAN_COST, CAST(NULL AS NUMBER) PLAN_CARDINALITY, CAST(NULL AS NUMBER) PLAN_BYTES, CAST(NULL AS NUMBER) PLAN_TIME, CAST(NULL AS VARCHAR2(256)) PLAN_PARTITION_START, CAST(NULL AS VARCHAR2(256)) PLAN_PARTITION_STOP, CAST(NULL AS NUMBER) PLAN_CPU_COST, CAST(NULL AS NUMBER) PLAN_IO_COST, CAST(NULL AS NUMBER) PLAN_TEMP_SPACE, CAST(STARTS AS NUMBER) STARTS, CAST(OUTPUT_ROWS AS NUMBER) OUTPUT_ROWS, CAST(NULL AS NUMBER) IO_INTERCONNECT_BYTES, CAST(NULL AS NUMBER) PHYSICAL_READ_REQUESTS, CAST(NULL AS NUMBER) PHYSICAL_READ_BYTES, CAST(NULL AS NUMBER) PHYSICAL_WRITE_REQUESTS, CAST(NULL AS NUMBER) PHYSICAL_WRITE_BYTES, CAST(WORKAREA_MEM AS NUMBER) WORKAREA_MEM, CAST(WORKAREA_MAX_MEM AS NUMBER) WORKAREA_MAX_MEM, CAST(WORKAREA_TEMPSEG AS NUMBER) WORKAREA_TEMPSEG, CAST(WORKAREA_MAX_TEMPSEG AS NUMBER) WORKAREA_MAX_TEMPSEG, CAST(NULL AS NUMBER) OTHERSTAT_GROUP_ID, CAST(OTHERSTAT_1_ID AS NUMBER) OTHERSTAT_1_ID, CAST(NULL AS NUMBER) OTHERSTAT_1_TYPE, CAST(OTHERSTAT_1_VALUE AS NUMBER) OTHERSTAT_1_VALUE, CAST(OTHERSTAT_2_ID AS NUMBER) OTHERSTAT_2_ID, CAST(NULL AS NUMBER) OTHERSTAT_2_TYPE, CAST(OTHERSTAT_2_VALUE AS NUMBER) OTHERSTAT_2_VALUE, CAST(OTHERSTAT_3_ID AS NUMBER) OTHERSTAT_3_ID, CAST(NULL AS NUMBER) OTHERSTAT_3_TYPE, CAST(OTHERSTAT_3_VALUE AS NUMBER) OTHERSTAT_3_VALUE, CAST(OTHERSTAT_4_ID AS NUMBER) OTHERSTAT_4_ID, CAST(NULL AS NUMBER) OTHERSTAT_4_TYPE, CAST(OTHERSTAT_4_VALUE AS NUMBER) OTHERSTAT_4_VALUE, CAST(OTHERSTAT_5_ID AS NUMBER) OTHERSTAT_5_ID, CAST(NULL AS NUMBER) OTHERSTAT_5_TYPE, CAST(OTHERSTAT_5_VALUE AS NUMBER) OTHERSTAT_5_VALUE, CAST(OTHERSTAT_6_ID AS NUMBER) OTHERSTAT_6_ID, CAST(NULL AS NUMBER) OTHERSTAT_6_TYPE, CAST(OTHERSTAT_6_VALUE AS NUMBER) OTHERSTAT_6_VALUE, CAST(OTHERSTAT_7_ID AS NUMBER) OTHERSTAT_7_ID, CAST(NULL AS NUMBER) OTHERSTAT_7_TYPE, CAST(OTHERSTAT_7_VALUE AS NUMBER) OTHERSTAT_7_VALUE, CAST(OTHERSTAT_8_ID AS NUMBER) OTHERSTAT_8_ID, CAST(NULL AS NUMBER) OTHERSTAT_8_TYPE, CAST(OTHERSTAT_8_VALUE AS NUMBER) OTHERSTAT_8_VALUE, CAST(OTHERSTAT_9_ID AS NUMBER) OTHERSTAT_9_ID, CAST(NULL AS NUMBER) OTHERSTAT_9_TYPE, CAST(OTHERSTAT_9_VALUE AS NUMBER) OTHERSTAT_9_VALUE, CAST(OTHERSTAT_10_ID AS NUMBER) OTHERSTAT_10_ID, CAST(NULL AS NUMBER) OTHERSTAT_10_TYPE, CAST(OTHERSTAT_10_VALUE AS NUMBER) OTHERSTAT_10_VALUE, CAST(NULL AS VARCHAR(1)) OTHER_XML, CAST(NULL AS NUMBER) PLAN_OPERATION_INACTIVE, OUTPUT_BATCHES, SKIPPED_ROWS_COUNT, DB_TIME, USER_IO_WAIT_TIME, NULL OTHER_WAIT_TIME FROM SYS.ALL_VIRTUAL_SQL_PLAN_MONITOR )__"))) { LOG_ERROR("fail to set view_definition", K(ret)); } } diff --git a/src/share/inner_table/ob_inner_table_schema_def.py b/src/share/inner_table/ob_inner_table_schema_def.py index df16dd960f..6f237ca524 100644 --- a/src/share/inner_table/ob_inner_table_schema_def.py +++ b/src/share/inner_table/ob_inner_table_schema_def.py @@ -16982,10 +16982,10 @@ def_table_schema( CAST( NULL AS UNSIGNED) AS PHYSICAL_READ_BYTES, CAST( NULL AS UNSIGNED) AS PHYSICAL_WRITE_REQUESTS, CAST( NULL AS UNSIGNED) AS PHYSICAL_WRITE_BYTES, - CAST( NULL AS UNSIGNED) AS WORKAREA_MEM, - CAST( NULL AS UNSIGNED) AS WORKAREA_MAX_MEM, - CAST( NULL AS UNSIGNED) AS WORKAREA_TEMPSEG, - CAST( NULL AS UNSIGNED) AS WORKAREA_MAX_TEMPSEG, + CAST( WORKAREA_MEM AS UNSIGNED) AS WORKAREA_MEM, + CAST( WORKAREA_MAX_MEM AS UNSIGNED) AS WORKAREA_MAX_MEM, + CAST( WORKAREA_TEMPSEG AS UNSIGNED) AS WORKAREA_TEMPSEG, + CAST( WORKAREA_MAX_TEMPSEG AS UNSIGNED) AS WORKAREA_MAX_TEMPSEG, CAST( NULL AS UNSIGNED) AS OTHERSTAT_GROUP_ID, OTHERSTAT_1_ID, CAST(NULL AS UNSIGNED) AS OTHERSTAT_1_TYPE, @@ -57040,10 +57040,10 @@ def_table_schema( CAST(NULL AS NUMBER) PHYSICAL_READ_BYTES, CAST(NULL AS NUMBER) PHYSICAL_WRITE_REQUESTS, CAST(NULL AS NUMBER) PHYSICAL_WRITE_BYTES, - CAST(NULL AS NUMBER) WORKAREA_MEM, - CAST(NULL AS NUMBER) WORKAREA_MAX_MEM, - CAST(NULL AS NUMBER) WORKAREA_TEMPSEG, - CAST(NULL AS NUMBER) WORKAREA_MAX_TEMPSEG, + CAST(WORKAREA_MEM AS NUMBER) WORKAREA_MEM, + CAST(WORKAREA_MAX_MEM AS NUMBER) WORKAREA_MAX_MEM, + CAST(WORKAREA_TEMPSEG AS NUMBER) WORKAREA_TEMPSEG, + CAST(WORKAREA_MAX_TEMPSEG AS NUMBER) WORKAREA_MAX_TEMPSEG, CAST(NULL AS NUMBER) OTHERSTAT_GROUP_ID, CAST(OTHERSTAT_1_ID AS NUMBER) OTHERSTAT_1_ID, CAST(NULL AS NUMBER) OTHERSTAT_1_TYPE, diff --git a/src/share/ob_autoincrement_param.h b/src/share/ob_autoincrement_param.h index ab71e6c324..17dfdd8bd1 100644 --- a/src/share/ob_autoincrement_param.h +++ b/src/share/ob_autoincrement_param.h @@ -191,6 +191,12 @@ OB_INLINE int64_t get_modify_autoinc_version(const int64_t &autoinc_version) return 0 == autoinc_version ? OB_INVALID_VERSION : autoinc_version; } +OB_INLINE int64_t get_auto_increment_cache_size(const int64_t table_cache_size, + const int64_t tenant_cache_size) +{ + return table_cache_size == 0 ? tenant_cache_size : table_cache_size; +} + }//end namespace share }//end namespace oceanbase #endif diff --git a/src/share/ob_autoincrement_service.cpp b/src/share/ob_autoincrement_service.cpp index 16f4c73389..a7e45969d3 100644 --- a/src/share/ob_autoincrement_service.cpp +++ b/src/share/ob_autoincrement_service.cpp @@ -309,6 +309,20 @@ int ObAutoincrementService::get_handle(AutoincParam ¶m, return ret; } +int ObAutoincrementService::get_handle(const ObSequenceSchema &schema, ObSequenceValue &nextval) +{ + ACTIVE_SESSION_FLAG_SETTER_GUARD(in_sequence_load); + int ret = OB_SUCCESS; + + if (OB_FAIL(global_autoinc_service_.get_sequence_next_value( + schema, nextval))) { + LOG_WARN("fail get value", K(ret)); + } else { + LOG_TRACE("succ to allocate cache handle", K(nextval), K(ret)); + } + return ret; +} + int ObAutoincrementService::get_handle_order(AutoincParam ¶m, CacheHandle *&handle) { int ret = OB_SUCCESS; @@ -345,12 +359,20 @@ int ObAutoincrementService::get_handle_order(AutoincParam ¶m, CacheHandle *& AutoincKey key(tenant_id, table_id, column_id); uint64_t table_auto_increment = param.autoinc_auto_increment_; + if (OB_UNLIKELY(offset > 1 && increment >= offset)) { + // If auto_increment_offset has been set, the base_value should be the maximum value of + // offset and table_auto_increment. + table_auto_increment = MAX(table_auto_increment, offset); + } uint64_t start_inclusive = 0; uint64_t end_inclusive = 0; uint64_t sync_value = 0; if (OB_UNLIKELY(table_auto_increment > max_value)) { - ret = OB_ERR_REACH_AUTOINC_MAX; - LOG_WARN("reach max autoinc", K(ret), K(table_auto_increment)); + // During the generation of the auto-increment column, if the user-specified value exceeds + // the maximum value, `OB_DATA_OUT_OF_RANGE` is returned, otherwise `OB_ERR_REACH_AUTOINC_MAX`. + ret = param.autoinc_auto_increment_ > max_value ? + OB_ERR_REACH_AUTOINC_MAX : OB_DATA_OUT_OF_RANGE; + LOG_WARN("reach max autoinc", K(ret), K(table_auto_increment), K(max_value)); } else if (OB_FAIL(global_autoinc_service_.get_value(key, offset, increment, max_value, table_auto_increment, batch_count, auto_increment_cache_size, autoinc_version, @@ -765,7 +787,8 @@ int ObAutoincrementService::try_lock_autoinc_row(const uint64_t &tenant_id, int ObAutoincrementService::clear_autoinc_cache_all(const uint64_t tenant_id, const uint64_t table_id, const uint64_t column_id, - const bool autoinc_is_order) + const bool autoinc_is_order, + const bool ignore_rpc_errors /*true*/) { int ret = OB_SUCCESS; if (OB_SUCC(ret)) { @@ -791,7 +814,7 @@ int ObAutoincrementService::clear_autoinc_cache_all(const uint64_t tenant_id, .by(tenant_id) .timeout(sync_timeout) .clear_autoinc_cache(arg))) { - if (is_rpc_error(ret) || autoinc_is_order) { + if (ignore_rpc_errors && (is_rpc_error(ret) || autoinc_is_order)) { // ignore time out and clear ordered auto increment cache error, go on LOG_WARN("rpc call time out, ignore the error", "server", iter->first, K(tenant_id), K(table_id), K(autoinc_is_order), K(ret)); @@ -1186,7 +1209,8 @@ int ObAutoincrementService::sync_insert_value_order(AutoincParam ¶m, value_to_sync = max_value; } if (OB_FAIL(global_autoinc_service_.local_push_to_global_value( - key, max_value, value_to_sync, autoinc_version, global_sync_value))) { + key, max_value, value_to_sync, autoinc_version, param.auto_increment_cache_size_, + global_sync_value))) { LOG_WARN("fail sync value to global", K(key), K(insert_value), K(ret)); } else if (NULL != cache_handle) { LOG_DEBUG("insert value, generate next val", @@ -1253,13 +1277,11 @@ int ObAutoincrementService::sync_insert_value_noorder(AutoincParam ¶m, if (insert_value <= ATOMIC_LOAD(&table_node->local_sync_)) { // do nothing } else { - const bool is_order = param.autoinc_mode_is_order_; - // For ORDER mode, the local cache_size is always 1, and the central(remote) cache_size - // is the configuration value. - const uint64_t local_cache_size = is_order ? 1 : param.auto_increment_cache_size_; + // For NOORDER mode, the central(remote) cache_size is the configuration value. + const uint64_t local_cache_size = MAX(1, param.auto_increment_cache_size_); uint64_t value_to_sync = calc_next_cache_boundary(insert_value, local_cache_size, max_value); if (OB_FAIL(distributed_autoinc_service_.local_push_to_global_value(key, max_value, - value_to_sync, autoinc_version, global_sync_value))) { + value_to_sync, autoinc_version, 0, global_sync_value))) { LOG_WARN("fail sync value to global", K(key), K(insert_value), K(ret)); } else { if (OB_FAIL(alloc_autoinc_try_lock(table_node->alloc_mutex_))) { @@ -1270,7 +1292,7 @@ int ObAutoincrementService::sync_insert_value_noorder(AutoincParam ¶m, atomic_update(table_node->last_refresh_ts_, ObTimeUtility::current_time()); } else { atomic_update(table_node->local_sync_, value_to_sync); - if (!is_order && OB_FAIL(sync_value_to_other_servers(param, value_to_sync))) { + if (OB_FAIL(sync_value_to_other_servers(param, value_to_sync))) { LOG_WARN("fail sync value to other servers", K(ret)); } } @@ -1731,8 +1753,10 @@ int ObInnerTableGlobalAutoIncrementService::local_push_to_global_value( const uint64_t max_value, const uint64_t insert_value, const int64_t &autoinc_version, + const int64_t cache_size, uint64_t &sync_value) { + UNUSED(cache_size); uint64_t seq_value = 0; // unused, * MUST * set seq_value to 0 here. return inner_table_proxy_.sync_autoinc_value(key, insert_value, max_value, autoinc_version, seq_value, sync_value); @@ -1786,6 +1810,13 @@ int ObRpcGlobalAutoIncrementService::get_value( start_inclusive, end_inclusive); } +int ObRpcGlobalAutoIncrementService::get_sequence_next_value( + const ObSequenceSchema &schema, + ObSequenceValue &nextval) +{ + return gais_client_.get_sequence_next_value(schema, nextval); +} + int ObRpcGlobalAutoIncrementService::get_sequence_value(const AutoincKey &key, const int64_t &autoinc_version, uint64_t &sequence_value) @@ -1808,9 +1839,12 @@ int ObRpcGlobalAutoIncrementService::local_push_to_global_value( const uint64_t max_value, const uint64_t value, const int64_t &autoinc_version, + const int64_t cache_size, uint64_t &global_sync_value) { - return gais_client_.local_push_to_global_value(key, max_value, value, autoinc_version, global_sync_value); + return gais_client_.local_push_to_global_value(key, max_value, value, autoinc_version, + cache_size, + global_sync_value); } int ObRpcGlobalAutoIncrementService::local_sync_with_global_value( @@ -1886,7 +1920,7 @@ int ObAutoIncInnerTableProxy::next_autoinc_value(const AutoincKey &key, column_id); if (sql_len >= OB_MAX_SQL_LENGTH || sql_len <= 0) { ret = OB_SIZE_OVERFLOW; - LOG_WARN("failed to format sql. size not enough"); + LOG_WARN("failed to format sql. size not enough", K(ret), K(sql_len)); } else { int64_t fetch_table_id = OB_INVALID_ID; { // make sure %res destructed before execute other sql in the same transaction @@ -1982,7 +2016,7 @@ int ObAutoIncInnerTableProxy::next_autoinc_value(const AutoincKey &key, int64_t affected_rows = 0; if (sql_len >= OB_MAX_SQL_LENGTH || sql_len <= 0) { ret = OB_SIZE_OVERFLOW; - LOG_WARN("failed to format sql. size not enough"); + LOG_WARN("failed to format sql. size not enough", K(ret), K(sql_len)); } else if (GCTX.is_standby_cluster() && OB_SYS_TENANT_ID != exec_tenant_id) { ret = OB_OP_NOT_ALLOW; LOG_WARN("can't write sys table now", K(ret), K(exec_tenant_id)); @@ -2043,7 +2077,7 @@ int ObAutoIncInnerTableProxy::get_autoinc_value(const AutoincKey &key, LOG_WARN("mysql proxy is null", K(ret)); } else if (sql_len >= OB_MAX_SQL_LENGTH || sql_len <= 0) { ret = OB_SIZE_OVERFLOW; - LOG_WARN("failed to format sql. size not enough"); + LOG_WARN("failed to format sql. size not enough", K(ret), K(sql_len)); } else if (OB_FAIL(sql_client_retry_weak.read(res, exec_tenant_id, sql))) { LOG_WARN(" failed to read data", K(ret)); } else if (NULL == (result = res.get_result())) { @@ -2200,7 +2234,7 @@ int ObAutoIncInnerTableProxy::sync_autoinc_value(const AutoincKey &key, LOG_WARN("mysql proxy is null", K(ret)); } else if (OB_FAIL(trans.start(mysql_proxy_, tenant_id, with_snap_shot))) { LOG_WARN("failed to start transaction", K(ret), K(tenant_id)); - } else { + } else { const uint64_t exec_tenant_id = tenant_id; const char *table_name = OB_ALL_AUTO_INCREMENT_TNAME; int64_t fetch_table_id = OB_INVALID_ID; @@ -2260,17 +2294,9 @@ int ObAutoIncInnerTableProxy::sync_autoinc_value(const AutoincKey &key, if (OB_SUCC(ret)) { uint64_t new_seq_value = 0; if (insert_value > sync_value) { - if (seq_value == 0) { - sync_value = std::max(fetch_seq_value - 1, insert_value); - new_seq_value = sync_value >= max_value ? max_value : sync_value + 1; - seq_value = new_seq_value; - } else { - // sequence value is vaild, use it to compute new sync/seq value - sync_value = std::max(seq_value - 1, insert_value); - new_seq_value = sync_value >= max_value ? max_value : - std::max(sync_value + 1, fetch_seq_value); - seq_value = sync_value >= max_value ? max_value : std::max(sync_value + 1, seq_value); - } + seq_value = std::max(fetch_seq_value, seq_value + 1); + sync_value = std::max(fetch_seq_value - 1, insert_value); + new_seq_value = sync_value >= max_value ? max_value : sync_value + 1; // if insert_value > global_sync // 2. update __all_sequence(may get global_sync) @@ -2312,6 +2338,149 @@ int ObAutoIncInnerTableProxy::sync_autoinc_value(const AutoincKey &key, LOG_WARN("fail to rollback transaction. ", K(err)); } } + } else { + seq_value = fetch_seq_value; + } + + // if transactin is started above(but do nothing), end it here + if (trans.is_started()) { + if (OB_SUCC(ret)) { + if (OB_FAIL(trans.end(true))) { + LOG_WARN("fail to commit transaction.", K(ret)); + } + } else { + int err = OB_SUCCESS; + if (OB_SUCCESS != (err = trans.end(false))) { + LOG_WARN("fail to rollback transaction. ", K(err)); + } + } + } + } + } + return ret; +} + +int ObAutoIncInnerTableProxy::read_and_push_inner_table(const AutoincKey &key, + const uint64_t max_value, + const uint64_t cache_end, + const int64_t autoinc_version, + bool &is_valid, + uint64_t &new_end) +{ + int ret = OB_SUCCESS; + const uint64_t tenant_id = key.tenant_id_; + const uint64_t table_id = key.table_id_; + const uint64_t column_id = key.column_id_; + is_valid = false; + ObMySQLTransaction trans; + ObSqlString sql; + bool with_snap_shot = true; + uint64_t fetch_seq_value = 0; + int64_t inner_autoinc_version = OB_INVALID_VERSION; + uint64_t sync_value = 0; + if (OB_ISNULL(mysql_proxy_)) { + ret = OB_NOT_INIT; + LOG_WARN("mysql proxy is null", K(ret)); + } else if (OB_FAIL(trans.start(mysql_proxy_, tenant_id, with_snap_shot))) { + LOG_WARN("failed to start transaction", K(ret), K(tenant_id)); + } else { + const uint64_t exec_tenant_id = tenant_id; + const char *table_name = OB_ALL_AUTO_INCREMENT_TNAME; + int64_t fetch_table_id = OB_INVALID_ID; + if (OB_FAIL(sql.assign_fmt(" SELECT sequence_value, truncate_version FROM %s WHERE tenant_id = %lu AND sequence_key = %lu" + " AND column_id = %lu FOR UPDATE", + table_name, + ObSchemaUtils::get_extract_tenant_id(exec_tenant_id, tenant_id), + ObSchemaUtils::get_extract_schema_id(exec_tenant_id, table_id), + column_id))) { + LOG_WARN("failed to assign sql", K(ret)); + } + if (OB_SUCC(ret)) { + SMART_VAR(ObMySQLProxy::MySQLResult, res) { + ObMySQLResult *result = NULL; + ObISQLClient *sql_client = &trans; + uint64_t sequence_table_id = OB_ALL_AUTO_INCREMENT_TID; + ObSQLClientRetryWeak sql_client_retry_weak(sql_client, + exec_tenant_id, + sequence_table_id); + if (OB_FAIL(sql_client_retry_weak.read(res, exec_tenant_id, sql.ptr()))) { + LOG_WARN("failed to read data", K(ret)); + } else if (NULL == (result = res.get_result())) { + LOG_WARN("failed to get result", K(ret)); + ret = OB_ERR_UNEXPECTED; + } else if (OB_FAIL(result->next())) { + LOG_WARN("failed to get next", K(ret)); + if (OB_ITER_END == ret) { + // auto-increment column has been deleted + ret = OB_SCHEMA_ERROR; + LOG_WARN("failed to get next", K(ret)); + } + } else if (OB_FAIL(result->get_uint(0l, fetch_seq_value))) { + LOG_WARN("failed to get int_value.", K(ret)); + } else if (OB_FAIL(result->get_int(1l, inner_autoinc_version))) { + LOG_WARN("failed to get inner_autoinc_version.", K(ret)); + } + if (OB_SUCC(ret)) { + int tmp_ret = OB_SUCCESS; + if (OB_ITER_END != (tmp_ret = result->next())) { + if (OB_SUCCESS == tmp_ret) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("more than one row", K(ret), K(tenant_id), K(table_id), K(column_id)); + } else { + ret = tmp_ret; + LOG_WARN("fail to iter next row", K(ret), K(tenant_id), K(table_id), K(column_id)); + } + } + } + } + } + if (OB_SUCC(ret)) { + if (autoinc_version != inner_autoinc_version) { + is_valid = false; + } else if (cache_end == fetch_seq_value && cache_end == max_value) { + // the column reach max value, keep the maximum value unchanged + is_valid = true; + new_end = max_value; + } else if (cache_end == fetch_seq_value - 1) { + // The cache is continuous and the verification passes. + is_valid = true; + uint64_t new_seq_value = fetch_seq_value; + if (fetch_seq_value >= max_value) { + new_end = max_value; + } else { + new_end = fetch_seq_value; + new_seq_value += 1; + // push new seq value to inner table + int64_t affected_rows = 0; + if (OB_FAIL(sql.assign_fmt( + "UPDATE %s SET sequence_value = %lu, gmt_modified = now(6) " + "WHERE tenant_id=%lu AND sequence_key=%lu AND column_id=%lu AND truncate_version=%ld", + table_name, new_seq_value, + OB_INVALID_TENANT_ID, table_id, column_id, inner_autoinc_version))) { + LOG_WARN("failed to assign sql", K(ret)); + } else if (GCTX.is_standby_cluster() && OB_SYS_TENANT_ID != exec_tenant_id) { + ret = OB_OP_NOT_ALLOW; + LOG_WARN("can't write sys table now", K(ret), K(exec_tenant_id)); + } else if (OB_FAIL((trans.write(exec_tenant_id, sql.ptr(), affected_rows)))) { + LOG_WARN("failed to execute", K(sql), K(ret)); + } else if (!is_single_row(affected_rows)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected error", K(affected_rows), K(ret)); + } else { + } + + // commit transaction or rollback + if (OB_SUCC(ret)) { + if (OB_FAIL(trans.end(true))) { + LOG_WARN("fail to commit transaction.", K(ret)); + } + } else { + int err = OB_SUCCESS; + if (OB_SUCCESS != (err = trans.end(false))) { + LOG_WARN("fail to rollback transaction. ", K(err)); + } + } + } } // if transactin is started above(but do nothing), end it here diff --git a/src/share/ob_autoincrement_service.h b/src/share/ob_autoincrement_service.h index c0da659c7d..8ca7b67933 100644 --- a/src/share/ob_autoincrement_service.h +++ b/src/share/ob_autoincrement_service.h @@ -226,6 +226,12 @@ public: const int64_t autoinc_version, uint64_t &seq_value, uint64_t &sync_value); + int read_and_push_inner_table(const AutoincKey &key, + const uint64_t max_value, + const uint64_t cache_end, + const int64_t autoinc_version, + bool &is_valid, + uint64_t &new_end); private: int check_inner_autoinc_version(const int64_t &request_autoinc_version, const int64_t &inner_autoinc_version, @@ -256,15 +262,17 @@ public: const int64_t &autoinc_version, uint64_t &sync_value, uint64_t &start_inclusive, - uint64_t &end_inclusive) override; + uint64_t &end_inclusive) override final; - virtual int get_sequence_value(const AutoincKey &key, const int64_t &autoinc_version, uint64_t &sequence_value) override; + virtual int get_sequence_value(const AutoincKey &key, + const int64_t &autoinc_version, + uint64_t &sequence_value) override final; virtual int get_auto_increment_values( const uint64_t tenant_id, const common::ObIArray &autoinc_keys, const common::ObIArray &autoinc_versions, - common::hash::ObHashMap &seq_values) override; + common::hash::ObHashMap &seq_values) override final; // when we push local value to global, we may find in global end that the local value // is obsolete. we will piggy back the larger global value to caller via global_sync_value, @@ -274,9 +282,12 @@ public: const uint64_t max_value, const uint64_t value, const int64_t &autoinc_version, - uint64_t &global_sync_value) override; + const int64_t cache_size, + uint64_t &global_sync_value) override final; - virtual int local_sync_with_global_value(const AutoincKey &key, const int64_t &autoinc_version, uint64_t &value) override; + virtual int local_sync_with_global_value(const AutoincKey &key, + const int64_t &autoinc_version, + uint64_t &value) override final; private: ObAutoIncInnerTableProxy inner_table_proxy_; }; @@ -301,15 +312,17 @@ public: const int64_t &autoinc_version, uint64_t &sync_value, uint64_t &start_inclusive, - uint64_t &end_inclusive) override; + uint64_t &end_inclusive) override final; - virtual int get_sequence_value(const AutoincKey &key, const int64_t &autoinc_version, uint64_t &sequence_value) override; + virtual int get_sequence_value(const AutoincKey &key, + const int64_t &autoinc_version, + uint64_t &sequence_value) override final; virtual int get_auto_increment_values( const uint64_t tenant_id, const common::ObIArray &autoinc_keys, const common::ObIArray &autoinc_versions, - common::hash::ObHashMap &seq_values) override; + common::hash::ObHashMap &seq_values) override final; // when we push local value to global, we may find in global end that the local value // is obsolete. we will piggy back the larger global value to caller via global_sync_value, @@ -319,11 +332,17 @@ public: const uint64_t max_value, const uint64_t value, const int64_t &autoinc_version, - uint64_t &global_sync_value) override; + const int64_t cache_size, + uint64_t &global_sync_value) override final; - virtual int local_sync_with_global_value(const AutoincKey &key, const int64_t &autoinc_version, uint64_t &value) override; + virtual int local_sync_with_global_value(const AutoincKey &key, + const int64_t &autoinc_version, + uint64_t &value) override final; int clear_global_autoinc_cache(const AutoincKey &key); + ObGAISRequestRpc *get_gais_request_rpc() { return &gais_request_rpc_; } + + int get_sequence_next_value(const ObSequenceSchema &schema, ObSequenceValue &nextval); private: bool is_inited_; @@ -353,6 +372,7 @@ public: share::schema::ObMultiVersionSchemaService *schema_service, rpc::frame::ObReqTransport *req_transport); int get_handle(AutoincParam ¶m, CacheHandle *&handle); + int get_handle(const ObSequenceSchema &schema, ObSequenceValue &nextval); void release_handle(CacheHandle *&handle); int sync_insert_value_global(AutoincParam ¶m); @@ -372,7 +392,8 @@ public: int clear_autoinc_cache_all(const uint64_t tenant_id, const uint64_t table_id, const uint64_t column_id, - const bool autoinc_mode_is_order); + const bool autoinc_mode_is_order, + const bool ignore_rpc_errors = true); int clear_autoinc_cache(const obrpc::ObAutoincSyncArg &arg); int get_sequence_value(const uint64_t tenant_id, @@ -417,6 +438,10 @@ public: const uint64_t offset, const uint64_t increment, uint64_t &prev_value); + ObGAISRequestRpc* get_gais_request_rpc() + { return global_autoinc_service_.get_gais_request_rpc(); } + static uint64_t get_max_value(const common::ObObjType type); + private: int get_handle_order(AutoincParam ¶m, CacheHandle *&handle); int get_handle_noorder(AutoincParam ¶m, CacheHandle *&handle); @@ -426,7 +451,6 @@ private: const uint64_t value_to_sync); private: - uint64_t get_max_value(const common::ObObjType type); int get_table_node(const AutoincParam ¶m, TableNode *&table_node); int fetch_table_node(const AutoincParam ¶m, TableNode *table_node, diff --git a/src/share/ob_gais_client.cpp b/src/share/ob_gais_client.cpp index e2a52adc68..abc0d7fb6e 100644 --- a/src/share/ob_gais_client.cpp +++ b/src/share/ob_gais_client.cpp @@ -186,6 +186,7 @@ int ObGAISClient::local_push_to_global_value(const AutoincKey &key, const uint64_t max_value, const uint64_t local_sync_value, const int64_t &autoinc_version, + const int64_t cache_size, uint64_t &global_sync_value) { int ret = OB_SUCCESS; @@ -200,7 +201,8 @@ int ObGAISClient::local_push_to_global_value(const AutoincKey &key, if (OB_FAIL(get_leader_(tenant_id, leader))) { LOG_WARN("get leader fail", K(ret)); (void)refresh_location_(tenant_id); - } else if (OB_FAIL(msg.init(key, local_sync_value, max_value, self_, autoinc_version))) { + } else if (OB_FAIL(msg.init(key, local_sync_value, max_value, self_, autoinc_version, + cache_size))) { LOG_WARN("fail to init request msg", KR(ret), K(key), K(autoinc_version)); } else if (OB_UNLIKELY(!msg.is_valid())) { ret = OB_INVALID_ARGUMENT; @@ -277,6 +279,40 @@ int ObGAISClient::clear_global_autoinc_cache(const AutoincKey &key) return ret; } +int ObGAISClient::get_sequence_next_value(const schema::ObSequenceSchema &schema, + ObSequenceValue &nextval) +{ + int ret = OB_SUCCESS; + const uint64_t tenant_id = schema.get_tenant_id(); + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not inited", K(ret)); + } else { + ObGAISNextSequenceValReq msg; + ObGAISNextSequenceValRpcResult rpc_result; + ObAddr leader; + if (OB_FAIL(get_leader_(tenant_id, leader))) { + LOG_WARN("get leader fail", K(ret)); + (void)refresh_location_(tenant_id); + } else if (OB_FAIL(msg.init(schema, self_))) { + LOG_WARN("fail to init request msg", K(ret)); + } else if (OB_UNLIKELY(!msg.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(msg)); + } else if (OB_UNLIKELY(OB_ISNULL(gais_request_rpc_))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("gais request rpc is null", K(ret)); + }else if (OB_FAIL(gais_request_rpc_->next_sequence_val(leader, msg, rpc_result))) { + LOG_WARN("handle gais request failed", K(ret), K(msg), K(rpc_result)); + (void)refresh_location_(tenant_id); + } else { + nextval.assign(rpc_result.nextval_); + LOG_DEBUG("handle gais success", K(rpc_result)); + } + } + return ret; +} + int ObGAISClient::get_leader_(const uint64_t tenant_id, ObAddr &leader) { int ret = gais_cache_leader_map_.get_refactored(tenant_id, leader); diff --git a/src/share/ob_gais_client.h b/src/share/ob_gais_client.h index 76ea2cc402..521ca7c062 100644 --- a/src/share/ob_gais_client.h +++ b/src/share/ob_gais_client.h @@ -52,10 +52,13 @@ public: const uint64_t max_value, const uint64_t local_sync_value, const int64_t &autoinc_version, + const int64_t cache_size, uint64_t &global_sync_value); int local_sync_with_global_value(const AutoincKey &key, const int64_t &autoinc_version, uint64_t &global_sync_value); int clear_global_autoinc_cache(const AutoincKey &key); + int get_sequence_next_value(const schema::ObSequenceSchema &schema, ObSequenceValue &nextval); + private: int get_leader_(const uint64_t tenant_id, common::ObAddr &leader); int refresh_location_(const uint64_t tenant_id); diff --git a/src/share/ob_gais_msg.cpp b/src/share/ob_gais_msg.cpp index f126b6ca35..56f78fcc6b 100644 --- a/src/share/ob_gais_msg.cpp +++ b/src/share/ob_gais_msg.cpp @@ -36,6 +36,44 @@ OB_SERIALIZE_MEMBER(ObGAISAutoIncKeyArg, autoinc_key_, sender_, autoinc_version_ OB_SERIALIZE_MEMBER(ObGAISPushAutoIncValReq, autoinc_key_, base_value_, max_value_, sender_, autoinc_version_, cache_size_); +OB_DEF_SERIALIZE(ObGAISBroadcastAutoIncCacheReq) +{ + int ret = OB_SUCCESS; + OB_UNIS_ENCODE(tenant_id_); + OB_UNIS_ENCODE(buf_size_); + if (OB_SUCC(ret)) { + if (pos + buf_size_ > buf_len) { + ret = OB_BUF_NOT_ENOUGH; + } else { + MEMCPY(buf + pos, buf_, buf_size_); + pos += buf_size_; + } + } + return ret; +} + +OB_DEF_DESERIALIZE(ObGAISBroadcastAutoIncCacheReq) +{ + int ret = OB_SUCCESS; + OB_UNIS_DECODE(tenant_id_); + OB_UNIS_DECODE(buf_size_); + if (OB_SUCC(ret)) { + buf_ = buf + pos; + pos += buf_size_; + } + return ret; +} + +OB_DEF_SERIALIZE_SIZE(ObGAISBroadcastAutoIncCacheReq) +{ + int64_t len = 0; + OB_UNIS_ADD_LEN(tenant_id_); + OB_UNIS_ADD_LEN(buf_size_); + len += buf_size_; + return len; +} +OB_SERIALIZE_MEMBER(ObGAISNextSequenceValReq, schema_, sender_); + int ObGAISNextAutoIncValReq::init(const AutoincKey &autoinc_key, const uint64_t offset, const uint64_t increment, @@ -85,7 +123,8 @@ int ObGAISPushAutoIncValReq::init(const AutoincKey &autoinc_key, const uint64_t base_value, const uint64_t max_value, const common::ObAddr &sender, - const int64_t &autoinc_version) + const int64_t &autoinc_version, + const int64_t cache_size) { int ret = OB_SUCCESS; if (!is_valid_tenant_id(autoinc_key.tenant_id_) || @@ -98,6 +137,35 @@ int ObGAISPushAutoIncValReq::init(const AutoincKey &autoinc_key, max_value_ = max_value; sender_ = sender; autoinc_version_ = get_modify_autoinc_version(autoinc_version); + cache_size_ = cache_size; + } + return ret; +} + +int ObGAISNextSequenceValReq::init(const schema::ObSequenceSchema &schema, + const common::ObAddr &sender) +{ + int ret = OB_SUCCESS; + if (!is_valid_tenant_id(schema.get_tenant_id()) || !sender.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(schema), K(sender)); + } else if (OB_FAIL(schema_.assign(schema))){ + LOG_WARN("fail to init schemar_", K(ret)); + } else { + sender_ = sender; + } + return ret; +} + +int ObGAISNextSequenceValReq::assign(const ObGAISNextSequenceValReq &src_req) +{ + int ret = OB_SUCCESS; + if (this != &src_req) { + if (OB_FAIL(schema_.assign(src_req.schema_))) { + LOG_WARN("fail assign schema_", K(src_req)); + } else { + sender_ = src_req.sender_; + } } return ret; } diff --git a/src/share/ob_gais_msg.h b/src/share/ob_gais_msg.h index 2102bbf1b7..74fcaeb129 100644 --- a/src/share/ob_gais_msg.h +++ b/src/share/ob_gais_msg.h @@ -96,11 +96,12 @@ public: const uint64_t base_value, const uint64_t max_value, const common::ObAddr &sender, - const int64_t &autoinc_version); + const int64_t &autoinc_version, + const int64_t cache_size); bool is_valid() const { return is_valid_tenant_id(autoinc_key_.tenant_id_) && max_value_ > 0 && base_value_ <= max_value_ - && sender_.is_valid() && autoinc_version_ >= OB_INVALID_VERSION; + && sender_.is_valid() && autoinc_version_ >= OB_INVALID_VERSION && cache_size_ >= 0; } TO_STRING_KV(K_(autoinc_key), K_(base_value), K_(max_value), K_(sender), K_(autoinc_version), K_(cache_size)); @@ -113,6 +114,53 @@ public: int64_t cache_size_; }; +struct ObGAISBroadcastAutoIncCacheReq +{ + OB_UNIS_VERSION(1); + +public: + ObGAISBroadcastAutoIncCacheReq() : tenant_id_(0), buf_(NULL), buf_size_(0) {} + + int init(const uint64_t tenant_id, const char *buf, const int64_t size) + { + tenant_id_ = tenant_id; + buf_ = buf; + buf_size_ = size; + return common::OB_SUCCESS; + } + + bool is_valid() const + { + return is_valid_tenant_id(tenant_id_) && NULL != buf_ && buf_size_ > 0; + } + + TO_STRING_KV(K_(tenant_id), KP_(buf), K_(buf_size)); + uint64_t tenant_id_; + const char *buf_; + int64_t buf_size_; +}; + +/* Request for get next sequence value */ +struct ObGAISNextSequenceValReq +{ + OB_UNIS_VERSION(1); + +public: + ObGAISNextSequenceValReq() : schema_(), sender_() + {} + int init(const schema::ObSequenceSchema &schema, const common::ObAddr &sender); + int assign(const ObGAISNextSequenceValReq &src_req); + bool is_valid() const + { + return is_valid_tenant_id(schema_.get_tenant_id()) && schema_.get_sequence_id() != OB_INVALID_ID + && schema_.get_cache_size() > static_cast(0) && sender_.is_valid(); + } + TO_STRING_KV(K_(schema), K_(sender)); + + schema::ObSequenceSchema schema_; + common::ObAddr sender_; +}; + } // share } // oceanbase diff --git a/src/share/ob_gais_rpc.cpp b/src/share/ob_gais_rpc.cpp index 9739719f9a..026a35d1b5 100644 --- a/src/share/ob_gais_rpc.cpp +++ b/src/share/ob_gais_rpc.cpp @@ -45,6 +45,32 @@ OB_SERIALIZE_MEMBER(ObGAISNextValRpcResult, start_inclusive_, end_inclusive_, sy OB_SERIALIZE_MEMBER(ObGAISCurrValRpcResult, sequence_value_, sync_value_); +OB_DEF_SERIALIZE(ObGAISNextSequenceValRpcResult) +{ + int ret = OB_SUCCESS; + OB_UNIS_ENCODE(nextval_); + return ret; +} + +OB_DEF_DESERIALIZE(ObGAISNextSequenceValRpcResult) +{ + int ret = OB_SUCCESS; + share::ObSequenceValue nextval; + OB_UNIS_DECODE(nextval); + // deep copy is needed to ensure that the memory of nextval_ will not be reclaimed + if (OB_SUCC(ret) && OB_FAIL(nextval_.assign(nextval))) { + LOG_WARN("fail to assign nextval", K(ret)); + } + return ret; +} + +OB_DEF_SERIALIZE_SIZE(ObGAISNextSequenceValRpcResult) +{ + int64_t len = 0; + OB_UNIS_ADD_LEN(nextval_); + return len; +} + int ObGAISNextValRpcResult::init(const uint64_t start_inclusive, const uint64_t end_inclusive, const uint64_t sync_value) { @@ -114,6 +140,46 @@ int ObGAISClearAutoIncCacheP::process() return ret; } +int ObGAISBroadcastAutoIncCacheP::process() +{ + int ret = OB_SUCCESS; + ObGlobalAutoIncService *gais = nullptr; + const uint64_t tenant_id = arg_.tenant_id_; + if (OB_ISNULL(gais = MTL(ObGlobalAutoIncService *))) { + // ignore if tenant service not in this server + } else if (OB_FAIL(gais->receive_global_autoinc_cache(arg_))) { + LOG_WARN("handle clear autoinc cache request failed", K(ret)); + } + return ret; +} + +int ObGAISNextSequenceP::process() +{ + int ret = OB_SUCCESS; + ObGlobalAutoIncService *gais = nullptr; + const uint64_t tenant_id = arg_.schema_.get_tenant_id(); + if (tenant_id != MTL_ID()) { + ret = OB_ERR_UNEXPECTED; + LOG_ERROR("tenant is not match", K(ret), K(tenant_id)); + } else if (OB_ISNULL(gais = MTL(ObGlobalAutoIncService *))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("global sequence service is null", K(ret)); + } else { + if (OB_FAIL(gais->handle_next_sequence_request(arg_, result_))) { + LOG_WARN("handle next sequence request failed", K(ret)); + } + } + return ret; +} + +int ObGAISNextSequenceValRpcResult::init(const share::ObSequenceValue nextval) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(nextval_.assign(nextval))) { + LOG_WARN("fail to assign nextval", K(ret)); + } + return ret; +} } // obrpc namespace share @@ -289,5 +355,76 @@ int ObGAISRequestRpc::clear_autoinc_cache(const ObAddr &server, const ObGAISAuto return ret; } +int ObGAISRequestRpc::broadcast_global_autoinc_cache(const ObGAISBroadcastAutoIncCacheReq &msg) +{ + int ret = OB_SUCCESS; + if (!is_inited_) { + ret = OB_NOT_INIT; + LOG_WARN("gais request rpc not inited", KR(ret)); + } else if (!msg.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(msg)); + } else { + ObZone null_zone; + ObSEArray server_list; + if (OB_FAIL(SVR_TRACER.get_alive_servers(null_zone, server_list))) { + LOG_WARN("fail to get alive server", K(ret)); + } else { + const uint64_t tenant_id = msg.tenant_id_; + const static int64_t BROADCAST_OP_TIMEOUT = 1000 * 1000; // 1s + for (int64_t i = 0; OB_SUCC(ret) && i < server_list.count(); i++) { + ObAddr &dest = server_list.at(i); + if (dest == self_) { + // ignore broadcast to self + } else if (OB_FAIL(rpc_proxy_->to(dest).by(tenant_id).timeout(BROADCAST_OP_TIMEOUT) + .broadcast_autoinc_cache(msg, NULL))) { + LOG_WARN("fail to broadcast autoinc cache to server", K(ret), K(msg), K(dest)); + ret = OB_SUCCESS; + } else { + LOG_DEBUG("broadcast autoinc cache success", K(dest), K(msg)); + } + } + } + } + return ret; +} + +int ObGAISRequestRpc::next_sequence_val(const common::ObAddr &server, + const ObGAISNextSequenceValReq &msg, + ObGAISNextSequenceValRpcResult &rpc_result) +{ + int ret = OB_SUCCESS; + const uint64_t timeout = THIS_WORKER.get_timeout_remain(); + if (!is_inited_) { + ret = OB_NOT_INIT; + LOG_WARN("gais request rpc not inited", KR(ret)); + } else if (!server.is_valid() || !msg.is_valid()) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", KR(ret), K(server), K(msg)); + } else if (server == self_) { + // Use local calls instead of rpc + ObGlobalAutoIncService *gais = nullptr; + const uint64_t tenant_id = msg.schema_.get_tenant_id(); + MTL_SWITCH(tenant_id) { + if (OB_ISNULL(gais = MTL(ObGlobalAutoIncService *))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("global autoinc service is null", K(ret)); + } else if (OB_FAIL(gais->handle_next_sequence_request(msg, rpc_result))) { + LOG_WARN("post local gais require autoinc request failed", KR(ret), K(server), K(msg)); + } else { + LOG_TRACE("post local require autoinc request success", K(msg), K(rpc_result)); + } + } + } else if (OB_UNLIKELY(OB_ISNULL(rpc_proxy_))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("rpc proxy is null", K(ret)); + }else if (OB_FAIL(rpc_proxy_->to(server).by(msg.schema_.get_tenant_id()).timeout(timeout).next_sequence_val(msg, rpc_result))) { + LOG_WARN("post require autoinc request failed", KR(ret), K(server), K(msg)); + } else { + LOG_TRACE("post remote require autoinc request success", K(server), K(msg), K(rpc_result)); + } + return ret; +} + } // share } // oceanbase diff --git a/src/share/ob_gais_rpc.h b/src/share/ob_gais_rpc.h index b4c6bc07c1..a4c183bab4 100644 --- a/src/share/ob_gais_rpc.h +++ b/src/share/ob_gais_rpc.h @@ -72,6 +72,17 @@ struct ObGAISCurrValRpcResult OB_UNIS_VERSION(1); }; +struct ObGAISNextSequenceValRpcResult +{ + ObGAISNextSequenceValRpcResult() : nextval_() {} + int init(const share::ObSequenceValue nextval_); + TO_STRING_KV(K_(nextval)); + share::ObSequenceValue nextval_; + + OB_UNIS_VERSION(1); +}; + + class ObGAISRpcProxy : public obrpc::ObRpcProxy { public: @@ -84,6 +95,11 @@ public: RPC_S(PR5 push_autoinc_val, OB_GAIS_PUSH_AUTO_INC_REQUEST, (share::ObGAISPushAutoIncValReq), uint64_t); RPC_S(PR5 clear_autoinc_cache, OB_GAIS_CLEAR_AUTO_INC_CACHE, (share::ObGAISAutoIncKeyArg)); + RPC_AP(PR3 broadcast_autoinc_cache, OB_GAIS_BROADCAST_AUTO_INC_CACHE, + (share::ObGAISBroadcastAutoIncCacheReq)); + + RPC_S(PR5 next_sequence_val, OB_GAIS_NEXT_SEQUENCE_REQUEST, + (share::ObGAISNextSequenceValReq), ObGAISNextSequenceValRpcResult); }; class ObGAISNextAutoIncP : public ObRpcProcessor< obrpc::ObGAISRpcProxy::ObRpc > @@ -126,6 +142,25 @@ private: DISALLOW_COPY_AND_ASSIGN(ObGAISClearAutoIncCacheP); }; +class ObGAISBroadcastAutoIncCacheP : public ObRpcProcessor< obrpc::ObGAISRpcProxy::ObRpc > +{ +public: + ObGAISBroadcastAutoIncCacheP() {} +protected: + int process(); +private: + DISALLOW_COPY_AND_ASSIGN(ObGAISBroadcastAutoIncCacheP); +}; +class ObGAISNextSequenceP : public ObRpcProcessor< obrpc::ObGAISRpcProxy::ObRpc > +{ +public: + ObGAISNextSequenceP() {} +protected: + int process(); +private: + DISALLOW_COPY_AND_ASSIGN(ObGAISNextSequenceP); +}; + } // obrpc namespace share @@ -146,6 +181,14 @@ public: int next_autoinc_val(const common::ObAddr &server, const ObGAISNextAutoIncValReq &msg, obrpc::ObGAISNextValRpcResult &rpc_result); + + /* + * Returns the next sequence value of specified key, + * and changes the current sequence value. + */ + int next_sequence_val(const common::ObAddr &server, + const ObGAISNextSequenceValReq &msg, + obrpc::ObGAISNextSequenceValRpcResult &rpc_result); /* * Returns the current auto-increment value of specified key. */ @@ -164,6 +207,8 @@ public: int clear_autoinc_cache(const common::ObAddr &server, const ObGAISAutoIncKeyArg &msg); + int broadcast_global_autoinc_cache(const ObGAISBroadcastAutoIncCacheReq &msg); + private: bool is_inited_; obrpc::ObGAISRpcProxy *rpc_proxy_; diff --git a/src/share/ob_global_autoinc_service.cpp b/src/share/ob_global_autoinc_service.cpp index d4f0573c55..b39364c1c6 100644 --- a/src/share/ob_global_autoinc_service.cpp +++ b/src/share/ob_global_autoinc_service.cpp @@ -28,6 +28,7 @@ #include "share/schema/ob_schema_struct.h" #include "share/schema/ob_schema_getter_guard.h" #include "share/schema/ob_multi_version_schema_service.h" +#include "src/share/sequence/ob_sequence_cache.h" namespace oceanbase { @@ -37,18 +38,21 @@ using namespace oceanbase::share; namespace share { -int ObAutoIncCacheNode::init(const uint64_t sequence_value, - const uint64_t last_available_value, +OB_SERIALIZE_MEMBER(ObAutoIncCacheNode, // FARM COMPAT WHITELIST + start_, end_, sync_value_, autoinc_version_); + +int ObAutoIncCacheNode::init(const uint64_t start, + const uint64_t end, const uint64_t sync_value, const int64_t autoinc_version) { int ret = OB_SUCCESS; - if (sequence_value <= 0 || last_available_value < sequence_value || sync_value > sequence_value || autoinc_version < OB_INVALID_VERSION) { + if (start <= 0 || end < start || sync_value > start || autoinc_version < OB_INVALID_VERSION) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid argument", K(ret), K(sequence_value), K(last_available_value), K(sync_value)); + LOG_WARN("invalid argument", K(ret), K(start), K(end), K(sync_value)); } else { - sequence_value_ = sequence_value; - last_available_value_ = last_available_value; + start_ = start; + end_ = end; sync_value_ = sync_value; autoinc_version_ = autoinc_version; } @@ -60,58 +64,56 @@ bool ObAutoIncCacheNode::need_fetch_next_node(const uint64_t base_value, const uint64_t max_value) const { bool bret = false; - if (OB_UNLIKELY(last_available_value_ == max_value)) { + if (OB_UNLIKELY(end_ == max_value)) { bret = false; - } else if (OB_LIKELY(last_available_value_ >= desired_cnt)) { - uint64_t new_base_value = std::max(base_value, sequence_value_); - bret = new_base_value > last_available_value_ - desired_cnt + 1; + } else if (OB_LIKELY(end_ >= desired_cnt)) { + uint64_t new_base_value = std::max(base_value, start_); + bret = new_base_value > (end_ - desired_cnt + 1); } else { bret = true; } return bret; } -int ObAutoIncCacheNode::update_sequence_value(const uint64_t sequence_value) +int ObAutoIncCacheNode::with_new_start(const uint64_t new_start) { int ret = OB_SUCCESS; if (OB_UNLIKELY(!is_valid())) { ret = OB_NOT_INIT; LOG_WARN("update invalid cache is not allowed", K(ret)); - } else if (OB_UNLIKELY(sequence_value > last_available_value_ || - sequence_value < sequence_value_)) { + } else if (OB_UNLIKELY(new_start > end_ || new_start < start_)) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid argument", K(ret), K(sequence_value), - K(sequence_value_), K(last_available_value_)); + LOG_WARN("invalid argument", K(ret), K(new_start), K_(start), K_(end)); } else { - sequence_value_ = sequence_value; + start_ = new_start; } return ret; } -int ObAutoIncCacheNode::update_available_value(const uint64_t available_value) +int ObAutoIncCacheNode::with_new_end(const uint64_t new_end) { int ret = OB_SUCCESS; if (OB_UNLIKELY(!is_valid())) { ret = OB_NOT_INIT; LOG_WARN("update invalid cache is not allowed", K(ret)); - } else if (OB_UNLIKELY(available_value < last_available_value_)) { + } else if (OB_UNLIKELY(new_end < end_)) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid argument", K(ret), K(available_value), K(last_available_value_)); + LOG_WARN("invalid argument", K(ret), K(new_end), K_(end)); } else { - last_available_value_ = available_value; + end_ = new_end; } return ret; } -int ObAutoIncCacheNode::update_sync_value(const uint64_t sync_value) +int ObAutoIncCacheNode::with_sync_value(const uint64_t sync_value) { int ret = OB_SUCCESS; if (OB_UNLIKELY(!is_valid())) { ret = OB_NOT_INIT; LOG_WARN("update invalid cache is not allowed", K(ret)); - } else if (OB_UNLIKELY(sync_value > sequence_value_)) { + } else if (OB_UNLIKELY(sync_value > start_)) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid argument", K(ret), K(sync_value), K(sequence_value_)); + LOG_WARN("invalid argument", K(ret), K(sync_value), K_(start)); } else { sync_value_ = sync_value; } @@ -131,6 +133,10 @@ int ObGlobalAutoIncService::init(const ObAddr &addr, ObMySQLProxy *mysql_proxy) attr, attr))) { LOG_WARN("init autoinc_map_ failed", K(ret)); + } else if (OB_ISNULL(gais_request_rpc_ = + ObAutoincrementService::get_instance().get_gais_request_rpc())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("gais request rpc is null", K(ret), KP(gais_request_rpc_)); } else { for (int64_t i = 0; i < MUTEX_NUM; ++i) { op_mutex_[i].set_latch_id(common::ObLatchIds::AUTO_INCREMENT_GAIS_LOCK); @@ -157,6 +163,8 @@ void ObGlobalAutoIncService::destroy() ObSpinLockGuard lock(cache_ls_lock_); cache_ls_ = NULL; is_leader_ = false; + gais_request_rpc_ = NULL; + is_switching_ = false; is_inited_ = false; } @@ -180,7 +188,7 @@ int ObGlobalAutoIncService::handle_next_autoinc_request( lib::ObMutex &mutex = op_mutex_[key.hash() % MUTEX_NUM]; if (OB_UNLIKELY(!is_inited_)) { ret = OB_NOT_INIT; - LOG_WARN("gloabl service is not init", K(ret)); + LOG_WARN("global service is not init", K(ret)); } else if (OB_UNLIKELY(!request.is_valid())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(request)); @@ -193,49 +201,53 @@ int ObGlobalAutoIncService::handle_next_autoinc_request( LOG_WARN("fail to get lock", K(ret)); } else { ObAutoIncCacheNode cache_node; - int err = autoinc_map_.get_refactored(key, cache_node); + int err = autoinc_map_.get_refactored(key.table_id_, cache_node); const int64_t tenant_id = key.tenant_id_; const int64_t request_version = request.autoinc_version_; LOG_TRACE("begin handle req autoinc request", K(request), K(cache_node)); if (OB_UNLIKELY(OB_SUCCESS != err && OB_HASH_NOT_EXIST != err)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("failed to get seq value", K(ret), K(key)); + } else if (OB_UNLIKELY(cache_node.is_received())) { + ret = read_and_push_inner_table(key, request.max_value_, cache_node); + } + if (OB_FAIL(ret)) { } else if (OB_UNLIKELY(!cache_node.is_valid() || (request_version == cache_node.autoinc_version_ && cache_node.need_fetch_next_node( request.base_value_, desired_count, request.max_value_)))) { OZ(fetch_next_node_(request, cache_node)); } else if (OB_UNLIKELY(request_version > cache_node.autoinc_version_)) { - LOG_INFO("start to reset old global table node", K(tenant_id), K(key.table_id_), - K(request_version), K(cache_node.autoinc_version_)); + LOG_INFO("start to reset old global table node", K(key), K(request_version), + K(cache_node.autoinc_version_)); cache_node.reset(); OZ(fetch_next_node_(request, cache_node)); } else if (OB_UNLIKELY(request_version < cache_node.autoinc_version_)) { ret = OB_AUTOINC_CACHE_NOT_EQUAL; - LOG_WARN("request autoinc_version is less than autoinc_version_ in table_node, it should retry", KR(ret), K(tenant_id), K(key.table_id_), - K(request_version), K(cache_node.autoinc_version_)); + LOG_WARN("request autoinc_version is less than autoinc_version_ in table_node," + "it should retry", KR(ret), K(tenant_id), K(key), K(request_version), K(cache_node)); } if (OB_SUCC(ret)) { if (OB_UNLIKELY(!cache_node.is_valid())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("Unexpected cache node", K(ret), K(cache_node)); } else { - const uint64_t start_inclusive = std::max(cache_node.sequence_value_, request.base_value_); + const uint64_t start_inclusive = std::max(cache_node.start_, request.base_value_); const uint64_t max_value = request.max_value_; uint64_t end_inclusive = 0; if (max_value >= request.desired_cnt_ && start_inclusive <= max_value - request.desired_cnt_ + 1) { end_inclusive = start_inclusive + request.desired_cnt_ - 1; - if (OB_UNLIKELY(end_inclusive > cache_node.last_available_value_)) { + if (OB_UNLIKELY(end_inclusive > cache_node.end_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected value", K(ret), K(end_inclusive), K(cache_node)); - } else if (OB_UNLIKELY(end_inclusive == cache_node.last_available_value_)) { + } else if (OB_UNLIKELY(end_inclusive == cache_node.end_)) { // the cache node is run out cache_node.reset(); - } else if (OB_FAIL(cache_node.update_sequence_value(end_inclusive + 1))) { + } else if (OB_FAIL(cache_node.with_new_start(end_inclusive + 1))) { LOG_WARN("fail to update sequence value", K(ret), K(cache_node), K(end_inclusive)); } - } else if (OB_FAIL(cache_node.update_sequence_value(max_value))) { + } else if (OB_FAIL(cache_node.with_new_start(max_value))) { LOG_WARN("fail to update sequence value", K(ret), K(cache_node), K(max_value)); } else { end_inclusive = max_value; @@ -247,7 +259,7 @@ int ObGlobalAutoIncService::handle_next_autoinc_request( } if (OB_FAIL(result.init(start_inclusive, end_inclusive, sync_value))) { LOG_WARN("init result failed", K(ret), K(cache_node)); - } else if (OB_FAIL(autoinc_map_.set_refactored(key, cache_node, 1))) { + } else if (OB_FAIL(autoinc_map_.set_refactored(key.table_id_, cache_node, 1))) { LOG_WARN("set autoinc_map_ failed", K(ret)); } } @@ -270,7 +282,7 @@ int ObGlobalAutoIncService::handle_curr_autoinc_request(const ObGAISAutoIncKeyAr lib::ObMutex &mutex = op_mutex_[key.hash() % MUTEX_NUM]; if (OB_UNLIKELY(!is_inited_)) { ret = OB_NOT_INIT; - LOG_WARN("gloabl service is not init", K(ret)); + LOG_WARN("global service is not init", K(ret)); } else if (OB_UNLIKELY(!request.is_valid())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(request)); @@ -281,26 +293,31 @@ int ObGlobalAutoIncService::handle_curr_autoinc_request(const ObGAISAutoIncKeyAr } else { ObAutoIncCacheNode cache_node; const int64_t tenant_id = key.tenant_id_; - int err = autoinc_map_.get_refactored(key, cache_node); + int err = autoinc_map_.get_refactored(key.table_id_, cache_node); const int64_t request_version = request.autoinc_version_; LOG_TRACE("start handle get autoinc request", K(request), K(cache_node)); if (OB_UNLIKELY(OB_SUCCESS != err && OB_HASH_NOT_EXIST != err)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("failed to get seq value", K(ret), K(key)); } else if (is_leader - && OB_LIKELY(cache_node.is_valid()) + && OB_LIKELY(cache_node.is_valid() && !cache_node.is_received()) && request_version == cache_node.autoinc_version_) { // get autoinc values from cache - sequence_value = cache_node.sequence_value_; + sequence_value = cache_node.start_; sync_value = cache_node.sync_value_; // hash not exist, cache node is non-valid or service is not leader, // read value from inner table - } else if (OB_FAIL(read_value_from_inner_table_(key, request_version, sequence_value, sync_value))) { + } else if (OB_FAIL(read_value_from_inner_table_(key, request_version, sequence_value, + sync_value))) { LOG_WARN("fail to read value from inner table", KR(ret), K(tenant_id), K_(key.table_id)); + } else if (OB_UNLIKELY(cache_node.is_received()) && (sequence_value - 1 == cache_node.end_)) { + sequence_value = cache_node.start_; + sync_value = cache_node.sync_value_; } if (OB_SUCC(ret)) { if (OB_FAIL(result.init(sequence_value, sync_value))) { - LOG_WARN("failed to init result", KR(ret), K(tenant_id), K_(key.table_id), K(request_version), K(cache_node)); + LOG_WARN("failed to init result", KR(ret), K(tenant_id), K_(key.table_id), + K(request_version), K(cache_node)); } } mutex.unlock(); @@ -318,7 +335,7 @@ int ObGlobalAutoIncService::handle_push_autoinc_request( lib::ObMutex &mutex = op_mutex_[key.hash() % MUTEX_NUM]; if (OB_UNLIKELY(!is_inited_)) { ret = OB_NOT_INIT; - LOG_WARN("gloabl service is not init", K(ret)); + LOG_WARN("global service is not init", K(ret)); } else if (OB_UNLIKELY(!request.is_valid())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(request)); @@ -332,32 +349,45 @@ int ObGlobalAutoIncService::handle_push_autoinc_request( } else { ObAutoIncCacheNode cache_node; const int64_t tenant_id = key.tenant_id_; - int err = autoinc_map_.get_refactored(key, cache_node); + int err = autoinc_map_.get_refactored(key.table_id_, cache_node); const int64_t request_version = request.autoinc_version_; + const uint64_t insert_value = request.base_value_; LOG_TRACE("start handle push global autoinc request", K(request), K(cache_node)); if (OB_UNLIKELY(OB_SUCCESS != err && OB_HASH_NOT_EXIST != err)) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("failed to get seq value", K(ret), K(key)); + LOG_WARN("failed to get seq value", K(ret), K(key), K(err)); } else if (OB_UNLIKELY(OB_HASH_NOT_EXIST == err || (request_version == cache_node.autoinc_version_ - && cache_node.need_sync(request.base_value_)) + && cache_node.need_sync(insert_value)) // cache node is expired || (request_version > cache_node.autoinc_version_))) { - if (request_version > cache_node.autoinc_version_) { - cache_node.reset(); - } + cache_node.reset(); if (OB_FAIL(sync_value_to_inner_table_(request, cache_node, sync_value))) { LOG_WARN("sync to inner table failed", K(ret)); - } else if (OB_FAIL(autoinc_map_.set_refactored(key, cache_node, 1))) { + } else if (OB_FAIL(autoinc_map_.set_refactored(key.table_id_, cache_node, 1))) { LOG_WARN("set autoinc_map_ failed", K(ret)); } // old request just ignore } else if (OB_UNLIKELY(request_version < cache_node.autoinc_version_)) { ret = OB_AUTOINC_CACHE_NOT_EQUAL; - LOG_WARN("request autoinc_version is less than cache_node autoinc_version", KR(ret), K(tenant_id), K_(key.table_id), - K(request_version), K(cache_node.autoinc_version_)); + LOG_WARN("request autoinc_version is less than cache_node autoinc_version", KR(ret), + K(key), K(request_version), K(cache_node.autoinc_version_)); } else if (OB_LIKELY(request_version == cache_node.autoinc_version_)) { - sync_value = cache_node.sync_value_; + if (insert_value < cache_node.start_ && insert_value < cache_node.sync_value_) { + // insert value is too small and no need to update node + } else { + sync_value = MAX(MAX(insert_value, cache_node.sync_value_), cache_node.start_ - 1); + if (OB_UNLIKELY(cache_node.is_received()) && + OB_FAIL(read_and_push_inner_table(key, request.max_value_, cache_node))) { + LOG_WARN("fail to read and push inner table", K(ret), K(key), K(cache_node)); + } else if (cache_node.is_valid()) { + cache_node.start_ = sync_value + 1; + cache_node.sync_value_ = sync_value; + } + if (OB_SUCC(ret) && OB_FAIL(autoinc_map_.set_refactored(key.table_id_, cache_node, 1))) { + LOG_WARN("set autoinc_map_ failed", K(ret)); + } + } } mutex.unlock(); } @@ -372,7 +402,7 @@ int ObGlobalAutoIncService::handle_clear_autoinc_cache_request(const ObGAISAutoI lib::ObMutex &mutex = op_mutex_[key.hash() % MUTEX_NUM]; if (OB_UNLIKELY(!is_inited_)) { ret = OB_NOT_INIT; - LOG_WARN("gloabl service is not init", K(ret)); + LOG_WARN("global service is not init", K(ret)); } else if (OB_UNLIKELY(!request.is_valid())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid argument", K(ret), K(request)); @@ -385,7 +415,7 @@ int ObGlobalAutoIncService::handle_clear_autoinc_cache_request(const ObGAISAutoI LOG_WARN("fail to get lock", K(ret)); } else { LOG_TRACE("start clear autoinc cache request", K(request)); - if (OB_FAIL(autoinc_map_.erase_refactored(key))) { + if (OB_FAIL(autoinc_map_.erase_refactored(key.table_id_))) { LOG_WARN("fail to erase autoinc cache map key", K(ret)); } if (ret == OB_HASH_NOT_EXIST) { @@ -396,11 +426,24 @@ int ObGlobalAutoIncService::handle_clear_autoinc_cache_request(const ObGAISAutoI return ret; } +int ObGlobalAutoIncService::handle_next_sequence_request( + const ObGAISNextSequenceValReq &request, + obrpc::ObGAISNextSequenceValRpcResult &result) +{ + int ret = OB_SUCCESS; + ObSequenceCache *sequence_cache = &share::ObSequenceCache::get_instance(); + ObArenaAllocator allocator; + return sequence_cache->nextval(request.schema_, allocator ,result.nextval_); +} + int ObGlobalAutoIncService::check_leader_(const uint64_t tenant_id, bool &is_leader) { int ret = OB_SUCCESS; is_leader = ATOMIC_LOAD(&is_leader_); if (OB_LIKELY(is_leader)) { + } else if (ATOMIC_LOAD(&is_switching_)) { + ret = OB_NOT_MASTER; + LOG_WARN("service is switching to leader", K(ret), KP(this), K(*this)); } else { // try to get role from logstream ObRole role = ObRole::INVALID_ROLE; @@ -443,8 +486,8 @@ int ObGlobalAutoIncService::fetch_next_node_(const ObGAISNextAutoIncValReq &requ end_inclusive, sync_value))) { LOG_WARN("fail to require autoinc value from inner table", K(ret)); - } else if (OB_LIKELY(node.is_valid() && node.last_available_value_ == start_inclusive - 1)) { - if (OB_FAIL(node.update_available_value(end_inclusive))) { + } else if (OB_LIKELY(node.is_valid() && (node.end_ == start_inclusive - request.increment_))) { + if (OB_FAIL(node.with_new_end(end_inclusive))) { LOG_WARN("fail to update available value", K(ret), K(node), K(end_inclusive)); } else { LOG_TRACE("fetch next node done", K(request), K(node)); @@ -472,40 +515,213 @@ int ObGlobalAutoIncService::sync_value_to_inner_table_( { int ret = OB_SUCCESS; const uint64_t insert_value = request.base_value_; - uint64_t seq_value = node.is_valid() ? node.sequence_value_ : 0; const int64_t autoinc_version = request.autoinc_version_; + const uint64_t next_cache_boundary = + calc_next_cache_boundary(insert_value, request.cache_size_, request.max_value_); + uint64_t seq_value = insert_value; if (OB_FAIL(inner_table_proxy_.sync_autoinc_value(request.autoinc_key_, - insert_value, + next_cache_boundary, request.max_value_, autoinc_version, seq_value, sync_value))) { LOG_WARN("fail to sync autoinc value to inner table", K(ret)); - } else if (OB_LIKELY(node.is_valid())) { - if (seq_value > node.last_available_value_) { - // the node is expired. - node.reset(); - node.sync_value_ = sync_value; // update sync value for next sync - } else if (sync_value == request.max_value_) { - if (node.last_available_value_ != request.max_value_) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpected error", K(ret), K(node), K(request.max_value_)); - } else if (OB_FAIL(node.init(request.max_value_, request.max_value_, request.max_value_, autoinc_version))) { - LOG_WARN("fail to init node", K(ret), K(request.max_value_)); - } - } else if (OB_FAIL(node.update_sequence_value(seq_value))) { - LOG_WARN("fail to update sequence value", K(ret), K(seq_value)); - } else if (OB_FAIL(node.update_sync_value(sync_value))) { - LOG_WARN("fail to update sync value", K(ret), K(sync_value)); + } else if (insert_value == request.max_value_) { + if (OB_FAIL(node.init(request.max_value_, request.max_value_, + request.max_value_, autoinc_version))) { + LOG_WARN("fail to init node", K(ret), K(request.max_value_)); } } else { - node.reset(); - node.sync_value_ = sync_value; // update sync value for next sync + // updates directly without checking, this node may be invalid. + node.start_ = seq_value; + node.end_ = sync_value; + node.sync_value_ = seq_value - 1; + node.autoinc_version_ = autoinc_version; + } + return ret; +} + +int ObGlobalAutoIncService::inner_switch_to_follower() +{ + int ret = OB_SUCCESS; + const int64_t start_time_us = ObTimeUtility::current_time(); + ObMutexGuard lock(switching_mutex_); + LOG_INFO("start to switch to follower", KP(this), K(*this)); + ATOMIC_STORE(&is_switching_, true); + ATOMIC_STORE(&is_leader_, false); + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("global service is not init", K(ret)); + } else if (OB_FAIL(broadcast_global_autoinc_cache())) { + LOG_WARN("fail to broadcast global autoinc cache", K(ret)); } if (OB_SUCC(ret)) { - node.autoinc_version_ = autoinc_version > node.autoinc_version_ ? autoinc_version : node.autoinc_version_; + // If the broadcast is successful, all cache nodes will be updated to is_received, + // otherwise the map will be cleared directly. + } else { + // ob failed + int tmp_ret = clear(); + if (tmp_ret != OB_SUCCESS) { + LOG_WARN("fail to clear auto inc map", K(ret), K(tmp_ret), K(autoinc_map_.size())); + } } + ATOMIC_STORE(&is_switching_, false); + const int64_t cost_us = ObTimeUtility::current_time() - start_time_us; + LOG_INFO("global_autoinc service: switch_to_follower", K(*this), K(cost_us)); + return ret; +} +int ObGlobalAutoIncService::broadcast_global_autoinc_cache() +{ + int ret = OB_SUCCESS; + if (autoinc_map_.size() > 0) { + if (OB_FAIL(wait_all_requests_to_finish())) { + LOG_WARN("fail to wait all requests to finish", K(ret)); + } else if (autoinc_map_.size() > 0) { + const int64_t size = serialize_size_autoinc_cache(); + const uint64_t tenant_id = MTL_ID(); + ObMemAttr attr(tenant_id, ObModIds::OB_AUTOINCREMENT); + char *buffer = NULL; + int64_t pos = 0; + ObGAISBroadcastAutoIncCacheReq msg; + if (OB_ISNULL(buffer = static_cast(ob_malloc(size, attr)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to alloc cache buffer", K(ret), K(size)); + } else if (OB_FAIL(serialize_autoinc_cache(buffer, size, pos))) { + LOG_WARN("fail to serialize global autoinc cache", K(ret)); + } else if (OB_FAIL(msg.init(tenant_id, buffer, pos))) { + LOG_WARN("fail to init msg", K(ret), K(tenant_id), K(buffer), K(pos)); + } else if (OB_UNLIKELY(!msg.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(msg)); + } else if (OB_ISNULL(gais_request_rpc_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("global service request rpc is not init", K(ret)); + } else if (OB_FAIL(gais_request_rpc_->broadcast_global_autoinc_cache(msg))) { + LOG_WARN("broadcast gais request failed", K(ret), K(msg)); + } else { + LOG_INFO("succ to broadcast global autoinc cache", K(msg)); + } + if (NULL != buffer) { + ob_free(buffer); + buffer = NULL; + } + } + } + return ret; +} + +int ObGlobalAutoIncService::receive_global_autoinc_cache( + const ObGAISBroadcastAutoIncCacheReq &request) +{ + int ret = OB_SUCCESS; + ObMutexGuard lock(switching_mutex_); + ATOMIC_STORE(&is_switching_, true); + ATOMIC_STORE(&is_leader_, false); + int64_t pos = 0; + if (OB_UNLIKELY(!request.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(request)); + } else if (OB_UNLIKELY(request.tenant_id_ != MTL_ID())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret), K(request), K(MTL_ID())); + } else if (OB_FAIL(wait_all_requests_to_finish())) { + LOG_WARN("fail to wait all requests to finish", K(ret)); + } else if (OB_FAIL(deserialize_autoinc_cache(request.buf_, request.buf_size_, pos))) { + LOG_WARN("fail to deserialize global auto inc cache", K(ret), K(request)); + } + ATOMIC_STORE(&is_switching_, false); + return ret; +} + +int64_t ObGlobalAutoIncService::serialize_size_autoinc_cache() +{ + int64_t len = 0; + common::hash::ObHashMap::iterator iter = autoinc_map_.begin(); + const uint64_t count = autoinc_map_.size(); + OB_UNIS_ADD_LEN(count); + for (; iter != autoinc_map_.end(); ++iter) { + const uint64_t &table_id = iter->first; + const ObAutoIncCacheNode &cache_node = iter->second; + OB_UNIS_ADD_LEN(table_id); + OB_UNIS_ADD_LEN(cache_node); + } + return len; +} + +int ObGlobalAutoIncService::serialize_autoinc_cache(SERIAL_PARAMS) +{ + int ret = OB_SUCCESS; + common::hash::ObHashMap::iterator iter = autoinc_map_.begin(); + const uint64_t count = autoinc_map_.size(); + OB_UNIS_ENCODE(count); + for (; OB_SUCC(ret) && iter != autoinc_map_.end(); ++iter) { + const uint64_t &table_id = iter->first; + ObAutoIncCacheNode &cache_node = iter->second; + OB_UNIS_ENCODE(table_id); + OB_UNIS_ENCODE(cache_node); + cache_node.is_received_ = true; // mark local cache as received + } + return ret; +} + +int ObGlobalAutoIncService::deserialize_autoinc_cache(DESERIAL_PARAMS) +{ + int ret = OB_SUCCESS; + uint64_t count = 0; + uint64_t table_id = 0; + ObAutoIncCacheNode cache_node; + ret = autoinc_map_.clear(); + OB_UNIS_DECODE(count); + for (int64_t i = 0; OB_SUCC(ret) && i < count; ++i) { + cache_node.reset(); + OB_UNIS_DECODE(table_id); + OB_UNIS_DECODE(cache_node); + cache_node.is_received_ = true; + if (OB_SUCC(ret) && OB_FAIL(autoinc_map_.set_refactored(table_id, cache_node, 1))) { + LOG_WARN("fail to set map", K(ret), K(table_id), K(cache_node)); + } + } + return ret; +} + +int ObGlobalAutoIncService::wait_all_requests_to_finish() +{ + int ret = OB_SUCCESS; + for (int64_t i = 0; OB_SUCC(ret) && i < MUTEX_NUM; i++) { + // wait for all working threads to finish + if (OB_FAIL(op_mutex_[i].lock(BROADCAST_OP_TIMEOUT))) { + LOG_WARN("fail to lock mutex", K(ret), K(i)); + } else { + op_mutex_[i].unlock(); + } + } + return ret; +} + +int ObGlobalAutoIncService::read_and_push_inner_table(const AutoincKey &key, + const uint64_t max_value, + ObAutoIncCacheNode &received_node) +{ + int ret = OB_SUCCESS; + if (received_node.is_valid()) { + bool is_valid = false; + uint64_t new_end = received_node.end_; + if (OB_FAIL(inner_table_proxy_.read_and_push_inner_table(key, + max_value, + received_node.end_, + received_node.autoinc_version_, + is_valid, + new_end))) { + LOG_WARN("fail to read and push inner table", K(ret), K(key), K(received_node)); + } else if (!is_valid) { + received_node.reset(); + } else if (OB_FAIL(received_node.with_new_end(new_end))) { + LOG_WARN("fail to update node end", K(ret), K(new_end), K(received_node)); + } else { + received_node.is_received_ = false; + } + } return ret; } diff --git a/src/share/ob_global_autoinc_service.h b/src/share/ob_global_autoinc_service.h index bbced69b59..70ebcbf753 100644 --- a/src/share/ob_global_autoinc_service.h +++ b/src/share/ob_global_autoinc_service.h @@ -37,48 +37,47 @@ namespace share struct ObGAISNextAutoIncValReq; struct ObGAISAutoIncKeyArg; struct ObGAISPushAutoIncValReq; +struct ObGAISBroadcastAutoIncCacheReq; struct ObAutoIncCacheNode { - ObAutoIncCacheNode() : sequence_value_(0), last_available_value_(0), sync_value_(0), autoinc_version_(OB_INVALID_VERSION) {} - int init(const uint64_t sequence_value, - const uint64_t last_available_value, + OB_UNIS_VERSION(1); +public: + ObAutoIncCacheNode() : start_(0), end_(0), sync_value_(0), autoinc_version_(OB_INVALID_VERSION), + is_received_(false) {} + int init(const uint64_t start, + const uint64_t end, const uint64_t sync_value, const int64_t autoinc_version); - bool is_valid() const { - return sequence_value_ > 0 && last_available_value_ >= sequence_value_ && - sync_value_ <= sequence_value_; - } - bool is_continuous(const uint64_t next_sequence_val, - const uint64_t sync_val, - const uint64_t max_val) const + inline bool is_valid() const { - return !is_valid() || (sync_val == sync_value_ && - ((last_available_value_ == max_val && next_sequence_val == max_val) || - last_available_value_ == next_sequence_val - 1)); + return start_ > 0 && end_ >= start_ && sync_value_ <= start_; } inline bool need_fetch_next_node(const uint64_t base_value, const uint64_t desired_cnt, const uint64_t max_value) const; inline bool need_sync(const uint64_t new_sync_value) const { - return new_sync_value > sync_value_; + return new_sync_value > sync_value_ && new_sync_value > end_; } - int update_sequence_value(const uint64_t sequence_value); - int update_available_value(const uint64_t available_value); - int update_sync_value(const uint64_t sync_value); + inline bool is_received() const { return is_received_; } + int with_new_start(const uint64_t new_start); + int with_new_end(const uint64_t new_end); + int with_sync_value(const uint64_t sync_value); void reset() { - sequence_value_ = 0; - last_available_value_ = 0; + start_ = 0; + end_ = 0; sync_value_ = 0; autoinc_version_ = OB_INVALID_VERSION; + is_received_ = false; } - TO_STRING_KV(K_(sequence_value), K_(last_available_value), K_(sync_value), K_(autoinc_version)); + TO_STRING_KV(K_(start), K_(end), K_(sync_value), K_(autoinc_version), K_(is_received)); - uint64_t sequence_value_; // next auto_increment value can be used - uint64_t last_available_value_; // last available value in the cache + uint64_t start_; // next auto_increment value can be used + uint64_t end_; // last available value in the cache(included) uint64_t sync_value_; int64_t autoinc_version_; + bool is_received_; }; class ObGlobalAutoIncService : public logservice::ObIReplaySubHandler, @@ -87,11 +86,15 @@ class ObGlobalAutoIncService : public logservice::ObIReplaySubHandler, { public: ObGlobalAutoIncService() : is_inited_(false), is_leader_(false), - cache_ls_lock_(common::ObLatchIds::AUTO_INCREMENT_LEADER_LOCK), cache_ls_(NULL) {} + cache_ls_lock_(common::ObLatchIds::AUTO_INCREMENT_LEADER_LOCK), cache_ls_(NULL), + gais_request_rpc_(NULL), is_switching_(false), + switching_mutex_(common::ObLatchIds::AUTO_INCREMENT_LEADER_LOCK) + {} virtual ~ObGlobalAutoIncService() {} const static int MUTEX_NUM = 1024; const static int INIT_HASHMAP_SIZE = 1000; + const static int64_t BROADCAST_OP_TIMEOUT = 1000 * 1000; // 1000ms, for broadcast auto increment cache int init(const common::ObAddr &addr, common::ObMySQLProxy *mysql_proxy); static int mtl_init(ObGlobalAutoIncService *&gais); void destroy(); @@ -123,15 +126,25 @@ public: uint64_t &sync_value); int handle_clear_autoinc_cache_request(const ObGAISAutoIncKeyArg &request); + int receive_global_autoinc_cache(const ObGAISBroadcastAutoIncCacheReq &request); + + /* + * This method handles the request for getting next (batch) sequence value. + * If the cache can satisfy the request, use the sequence in the cache to return, + * otherwise, need to require sequence from inner table and fill it in the cache, + * and then consume the sequence in the cache. + */ + int handle_next_sequence_request(const ObGAISNextSequenceValReq &request, + obrpc::ObGAISNextSequenceValRpcResult &result); public: - void switch_to_follower_forcedly() { - ATOMIC_STORE(&is_leader_, false); - clear(); + void switch_to_follower_forcedly() + { + inner_switch_to_follower(); } - int switch_to_follower_gracefully() { - ATOMIC_STORE(&is_leader_, false); - return clear(); + int switch_to_follower_gracefully() + { + return inner_switch_to_follower(); } int resume_leader() { ATOMIC_STORE(&is_leader_, true); @@ -139,7 +152,7 @@ public: } int switch_to_leader() { ATOMIC_STORE(&is_leader_, true); - return clear(); + return common::OB_SUCCESS; } // for replay, do nothing @@ -174,6 +187,9 @@ public: cache_ls_ = ls_ptr; } + TO_STRING_KV(K_(is_inited), K_(is_leader), K_(self), K(autoinc_map_.size()), KP_(cache_ls), + K_(is_switching)); + private: int check_leader_(const uint64_t tenant_id, bool &is_leader); int fetch_next_node_(const ObGAISNextAutoIncValReq &request, ObAutoIncCacheNode &node); @@ -184,16 +200,45 @@ private: int sync_value_to_inner_table_(const ObGAISPushAutoIncValReq &request, ObAutoIncCacheNode &node, uint64_t &sync_value); + static uint64_t calc_next_cache_boundary(const uint64_t insert_value, + const uint64_t cache_size, + const uint64_t max_value) + { + uint64_t next_cache_boundary = 0; + if (max_value < cache_size || insert_value > max_value - cache_size) { + next_cache_boundary = max_value; + } else { + next_cache_boundary = insert_value + cache_size; + } + return next_cache_boundary; + } + int inner_switch_to_follower(); + int broadcast_global_autoinc_cache(); + int64_t serialize_size_autoinc_cache(); + int serialize_autoinc_cache(SERIAL_PARAMS); + int deserialize_autoinc_cache(DESERIAL_PARAMS); + int wait_all_requests_to_finish(); + /* + * The function will check whether the data of the received node and inner table are consistent. + * If they are consistent, the inner table will be pushed up by increasing seq_value. + * Otherwise, the node will be set as invalid. + */ + int read_and_push_inner_table(const AutoincKey &key, + const uint64_t max_value, + ObAutoIncCacheNode &received_node); private: bool is_inited_; bool is_leader_; common::ObAddr self_; share::ObAutoIncInnerTableProxy inner_table_proxy_; - common::hash::ObHashMap autoinc_map_; + common::hash::ObHashMap autoinc_map_; // table_id -> node common::ObSpinLock cache_ls_lock_; storage::ObLS *cache_ls_; lib::ObMutex op_mutex_[MUTEX_NUM]; + ObGAISRequestRpc* gais_request_rpc_; + bool is_switching_; + lib::ObMutex switching_mutex_; }; } // share diff --git a/src/share/ob_i_global_autoincrement_service.h b/src/share/ob_i_global_autoincrement_service.h index 9a22a1bcbd..e12e142950 100644 --- a/src/share/ob_i_global_autoincrement_service.h +++ b/src/share/ob_i_global_autoincrement_service.h @@ -55,6 +55,7 @@ public: const uint64_t max_value, const uint64_t insert_value, const int64_t &autoinc_version, + const int64_t cache_size, uint64_t &global_sync_value) = 0; virtual int local_sync_with_global_value(const AutoincKey &key, const int64_t &autoinc_version, uint64_t &value) = 0; diff --git a/src/share/ob_scanner.cpp b/src/share/ob_scanner.cpp index 47831b1779..242c0a50da 100644 --- a/src/share/ob_scanner.cpp +++ b/src/share/ob_scanner.cpp @@ -50,7 +50,9 @@ ObScanner::ObScanner(const char *label /*= ObModIds::OB_NEW_SCANNER*/, implicit_cursors_(inner_allocator_), datum_store_(label), rcode_(), - fb_info_() + fb_info_(), + memstore_read_row_count_(0), + ssstore_read_row_count_(0) { UNUSED(allocator); } @@ -80,7 +82,9 @@ ObScanner::ObScanner(ObIAllocator &allocator, implicit_cursors_(allocator), datum_store_(label, &allocator), rcode_(), - fb_info_() + fb_info_(), + memstore_read_row_count_(0), + ssstore_read_row_count_(0) { } @@ -118,6 +122,8 @@ void ObScanner::reuse() is_result_accurate_ = true; trans_result_.reset(); fb_info_.reset(); + memstore_read_row_count_ = 0; + ssstore_read_row_count_ = 0; } int ObScanner::init(int64_t mem_size_limit /*= DEFAULT_MAX_SERIALIZE_SIZE*/) @@ -163,6 +169,8 @@ void ObScanner::reset() is_result_accurate_ = true; trans_result_.reset(); fb_info_.reset(); + memstore_read_row_count_ = 0; + ssstore_read_row_count_ = 0; } int ObScanner::add_row(const ObNewRow &row) @@ -258,6 +266,8 @@ int ObScanner::assign(const ObScanner &other) STRNCPY(rcode_.msg_, other.rcode_.msg_, common::MAX_SQL_ERR_MSG_LENGTH - 1); rcode_.rcode_ = other.rcode_.rcode_; OZ(fb_info_.assign(other.fb_info_)); + memstore_read_row_count_ = other.memstore_read_row_count_; + ssstore_read_row_count_ = other.ssstore_read_row_count_; return ret; } @@ -375,7 +385,9 @@ OB_DEF_SERIALIZE(ObScanner) rcode_.warnings_, tenant_id_, datum_store_, - fb_info_); + fb_info_, + memstore_read_row_count_, + ssstore_read_row_count_); return ret; } @@ -403,7 +415,9 @@ OB_DEF_SERIALIZE_SIZE(ObScanner) rcode_.warnings_, tenant_id_, datum_store_, - fb_info_); + fb_info_, + memstore_read_row_count_, + ssstore_read_row_count_); return len; } @@ -447,7 +461,10 @@ OB_DEF_DESERIALIZE(ObScanner) LOG_WARN("fail to write string", K(ret)); } } - OB_UNIS_DECODE(fb_info_); + LST_DO_CODE(OB_UNIS_DECODE, + fb_info_, + memstore_read_row_count_, + ssstore_read_row_count_); return ret; } diff --git a/src/share/ob_scanner.h b/src/share/ob_scanner.h index 6304cb7b66..9b4594b088 100644 --- a/src/share/ob_scanner.h +++ b/src/share/ob_scanner.h @@ -117,6 +117,13 @@ public: void set_err_code(int err_code) { rcode_.rcode_ = err_code; } int get_err_code() const { return rcode_.rcode_; } + + void set_memstore_read_row_count(int64_t memstore_read_row_count) { memstore_read_row_count_ = memstore_read_row_count; } + int64_t get_memstore_read_row_count() const { return memstore_read_row_count_; } + + void set_ssstore_read_row_count(int64_t ssstore_read_row_count) { ssstore_read_row_count_ = ssstore_read_row_count; } + int64_t get_ssstore_read_row_count() const { return ssstore_read_row_count_; } + void log_user_error_and_warn() const; const transaction::ObTxExecResult &get_trans_result() const { return trans_result_; } transaction::ObTxExecResult &get_trans_result() { return trans_result_; } @@ -159,7 +166,9 @@ public: K_(is_result_accurate), K_(trans_result), K_(implicit_cursors), - K_(rcode)); + K_(rcode), + K_(memstore_read_row_count), + K_(ssstore_read_row_count)); protected: ObRowStore row_store_; int64_t mem_size_limit_; /**< memory size of row store */ @@ -186,6 +195,9 @@ protected: sql::ObChunkDatumStore datum_store_; obrpc::ObRpcResultCode rcode_; sql::ObExecFeedbackInfo fb_info_; + int64_t memstore_read_row_count_; + int64_t ssstore_read_row_count_; + private: DISALLOW_COPY_AND_ASSIGN(ObScanner); }; diff --git a/src/share/rc/ob_tenant_base.h b/src/share/rc/ob_tenant_base.h index d635cde1b3..0af10357d3 100755 --- a/src/share/rc/ob_tenant_base.h +++ b/src/share/rc/ob_tenant_base.h @@ -311,12 +311,12 @@ using ObTableScanIteratorObjPool = common::ObServerObjectPool 0 && + OB_FAIL(databuff_printf(buf, buf_len, pos, "AUTO_INCREMENT_CACHE_SIZE = %ld ", + table_schema.get_auto_increment_cache_size()))) { + SHARE_SCHEMA_LOG(WARN, "fail to print table auto increment cache size", K(ret), + K(table_schema)); } } } @@ -2176,6 +2181,10 @@ int ObSchemaPrinter::print_table_definition_table_options( } else if (OB_FAIL(databuff_printf(buf, buf_len, pos, "AUTO_INCREMENT_MODE = '%s' ", table_schema.is_order_auto_increment_mode() ? "ORDER" : "NOORDER"))) { OB_LOG(WARN, "fail to print auto increment mode", K(ret), K(table_schema)); + } else if (table_schema.get_auto_increment_cache_size() > 0 && + OB_FAIL(databuff_printf(buf, buf_len, pos, "AUTO_INCREMENT_CACHE_SIZE = %ld ", + table_schema.get_auto_increment_cache_size()))) { + OB_LOG(WARN, "fail to print table auto increment cache size", K(ret), K(table_schema)); } } } diff --git a/src/share/schema/ob_schema_retrieve_utils.ipp b/src/share/schema/ob_schema_retrieve_utils.ipp index d48ddc292a..dc582a32e0 100644 --- a/src/share/schema/ob_schema_retrieve_utils.ipp +++ b/src/share/schema/ob_schema_retrieve_utils.ipp @@ -1501,6 +1501,8 @@ int ObSchemaRetrieveUtils::fill_table_schema( uint64_t, true, true/*ignore_column_error*/, COLUMN_GROUP_START_ID); EXTRACT_INT_FIELD_TO_CLASS_MYSQL_WITH_DEFAULT_VALUE(result, column_store, table_schema, bool, true, true/*ignore_column_error*/, false); + EXTRACT_INT_FIELD_TO_CLASS_MYSQL_WITH_DEFAULT_VALUE(result, auto_increment_cache_size, table_schema, + int64_t, true, true, 0); } if (OB_SUCC(ret) && OB_FAIL(fill_sys_table_lob_tid(table_schema))) { SHARE_SCHEMA_LOG(WARN, "fail to fill lob table id for inner table", K(ret), K(table_schema.get_table_id())); @@ -2891,6 +2893,9 @@ int ObSchemaRetrieveUtils::fill_sequence_schema( EXTRACT_BOOL_FIELD_TO_CLASS_MYSQL(result, order_flag, sequence_schema); bool ignore_column_error = false; EXTRACT_BOOL_FIELD_TO_CLASS_MYSQL_WITH_DEFAULT_VALUE(result, is_system_generated, sequence_schema, true, ignore_column_error, false); + ignore_column_error = true; + EXTRACT_INT_FIELD_TO_CLASS_MYSQL_WITH_DEFAULT_VALUE(result, flag, sequence_schema, uint64_t, + true, ignore_column_error, 0); } return ret; } diff --git a/src/share/schema/ob_sequence_sql_service.cpp b/src/share/schema/ob_sequence_sql_service.cpp index 56e93aa87c..a7c42403f1 100644 --- a/src/share/schema/ob_sequence_sql_service.cpp +++ b/src/share/schema/ob_sequence_sql_service.cpp @@ -24,6 +24,7 @@ #include "share/ob_srv_rpc_proxy.h" #include "observer/ob_server_struct.h" #include "observer/ob_srv_network_frame.h" +#include "share/ob_autoincrement_service.h" namespace oceanbase { @@ -215,6 +216,24 @@ int ObSequenceSqlService::replace_sequence(const ObSequenceSchema &sequence_sche || OB_FAIL(dml.add_column("schema_version", sequence_schema.get_schema_version())) || OB_FAIL(dml.add_gmt_modified())) { LOG_WARN("add column failed", K(ret)); + } else { + uint64_t compat_version = 0; + if (FAILEDx(GET_MIN_DATA_VERSION(tenant_id, compat_version))) { + LOG_WARN("fail to get data version", KR(ret), K(tenant_id)); + } else if (((compat_version < MOCK_DATA_VERSION_4_2_3_0) + || (compat_version >= DATA_VERSION_4_3_0_0 + && compat_version < DATA_VERSION_4_3_2_0)) + && sequence_schema.get_flag() != 0) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("not suppported flag != 0 when tenant's data version is below 4.2.3.0", + KR(ret)); + } else if ((compat_version >= MOCK_DATA_VERSION_4_2_3_0 + && compat_version < DATA_VERSION_4_3_0_0) + || (compat_version >= DATA_VERSION_4_3_2_0)) { + if (OB_FAIL(dml.add_column("flag", sequence_schema.get_flag()))) { + LOG_WARN("add flag column failed", K(ret)); + } + } } } else { // rename sequence if (OB_FAIL(dml.add_pk_column("tenant_id", ObSchemaUtils::get_extract_tenant_id( @@ -247,7 +266,6 @@ int ObSequenceSqlService::replace_sequence(const ObSequenceSchema &sequence_sche K(ret)); } } - if (OB_SUCC(ret)) { if (alter_start_with && OB_FAIL(alter_sequence_start_with(sequence_schema, *sql_client))) { LOG_WARN("alter sequence for start with failed", K(ret)); @@ -435,6 +453,20 @@ int ObSequenceSqlService::add_sequence(common::ObISQLClient &sql_client, SQL_COL_APPEND_VALUE(sql, values, sequence_schema.get_order_flag(), "order_flag", "%d"); SQL_COL_APPEND_VALUE(sql, values, sequence_schema.get_cycle_flag(), "cycle_flag", "%d"); SQL_COL_APPEND_VALUE(sql, values, sequence_schema.get_is_system_generated(), "is_system_generated", "%d"); + uint64_t compat_version = 0; + if (FAILEDx(GET_MIN_DATA_VERSION(tenant_id, compat_version))) { + LOG_WARN("fail to get data version", KR(ret), K(tenant_id)); + } else if (((compat_version < MOCK_DATA_VERSION_4_2_3_0) + || (compat_version >= DATA_VERSION_4_3_0_0 + && compat_version < DATA_VERSION_4_3_2_0)) + && sequence_schema.get_flag() != 0) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("not suppported flag != 0 when tenant's data version is below 4.2.3.0", KR(ret)); + } else if ((compat_version >= MOCK_DATA_VERSION_4_2_3_0 + && compat_version < DATA_VERSION_4_3_0_0) + || (compat_version >= DATA_VERSION_4_3_2_0)) { + SQL_COL_APPEND_VALUE(sql, values, sequence_schema.get_flag(), "flag", "%ld"); + } if (0 == STRCMP(tname[i], OB_ALL_SEQUENCE_OBJECT_HISTORY_TNAME)) { SQL_COL_APPEND_VALUE(sql, values, "false", "is_deleted", "%s"); } diff --git a/src/share/schema/ob_table_sql_service.cpp b/src/share/schema/ob_table_sql_service.cpp index 3aa37d5f43..2ab5478005 100644 --- a/src/share/schema/ob_table_sql_service.cpp +++ b/src/share/schema/ob_table_sql_service.cpp @@ -2994,6 +2994,11 @@ int ObTableSqlService::gen_table_dml( || 0 == strcasecmp(table.get_compress_func_name(), all_compressor_name[ObCompressorType::ZLIB_LITE_COMPRESSOR]))) { ret = OB_NOT_SUPPORTED; LOG_WARN("zlib_lite_1.0 not support before 4.3", K(ret), K(table)); + } else if ((data_version < MOCK_DATA_VERSION_4_2_3_0 || + (data_version >= DATA_VERSION_4_3_0_0 && data_version < DATA_VERSION_4_3_2_0)) + && OB_UNLIKELY(0 != table.get_auto_increment_cache_size())) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("auto increment cache size not support before 4.2.3", K(ret), K(table)); } else { if (data_version < DATA_VERSION_4_2_1_0 && (!table.get_ttl_definition().empty() || !table.get_kv_attributes().empty())) { @@ -3137,6 +3142,8 @@ int ObTableSqlService::gen_table_dml( && OB_FAIL(dml.add_column("max_used_column_group_id", table.get_max_used_column_group_id()))) || (data_version >= DATA_VERSION_4_3_0_0 && OB_FAIL(dml.add_column("column_store", table.is_column_store_supported()))) + || ((data_version >= DATA_VERSION_4_3_2_0 || (data_version < DATA_VERSION_4_3_0_0 && data_version >= MOCK_DATA_VERSION_4_2_3_0)) + && OB_FAIL(dml.add_column("auto_increment_cache_size", table.get_auto_increment_cache_size()))) ) { LOG_WARN("add column failed", K(ret)); } @@ -3172,6 +3179,11 @@ int ObTableSqlService::gen_table_options_dml( && OB_UNLIKELY(OB_DEFAULT_LOB_INROW_THRESHOLD != table.get_lob_inrow_threshold())) { ret = OB_NOT_SUPPORTED; LOG_WARN("lob_inrow_threshold not support before 4.2.1.2", K(ret), K(table)); + } else if ((data_version < MOCK_DATA_VERSION_4_2_3_0 + || (data_version >= DATA_VERSION_4_3_0_0 && data_version < DATA_VERSION_4_3_2_0)) + && (table.get_auto_increment_cache_size() != 0)) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("table auto_increment_cache_size not support before 4.2.3", K(ret), K(table)); } else {} if (OB_SUCC(ret)) { const ObPartitionOption &part_option = table.get_part_option(); @@ -3277,6 +3289,8 @@ int ObTableSqlService::gen_table_options_dml( && OB_FAIL(dml.add_column("max_used_column_group_id", table.get_max_used_column_group_id()))) || (data_version >= DATA_VERSION_4_3_0_0 && OB_FAIL(dml.add_column("column_store", table.is_column_store_supported()))) + || ((data_version >= DATA_VERSION_4_3_2_0 || (data_version < DATA_VERSION_4_3_0_0 && data_version >= MOCK_DATA_VERSION_4_2_3_0)) + && OB_FAIL(dml.add_column("auto_increment_cache_size", table.get_auto_increment_cache_size()))) ) { LOG_WARN("add column failed", K(ret)); } diff --git a/src/share/sequence/ob_sequence_dml_proxy.cpp b/src/share/sequence/ob_sequence_dml_proxy.cpp index c7127e8f68..a0fe3f33b8 100644 --- a/src/share/sequence/ob_sequence_dml_proxy.cpp +++ b/src/share/sequence/ob_sequence_dml_proxy.cpp @@ -116,15 +116,12 @@ int ObSequenceDMLProxy::next_batch( const ObNumber &increment_by = option.get_increment_by(); bool order_flag = option.get_order_flag(); bool cycle_flag = option.get_cycle_flag(); + ObSequenceCacheOrderMode cache_order_mode = option.get_cache_order_mode(); - if (true == order_flag) { - // 注意:性能很差!! - // 对于要求全局有序的场景, OB 只能强制要求每次都读 - // __all_sequence_value 表,使用行锁来保证全局顺序 - // 性能很差,但目前并没有更好的解决方案 - // - // 为了达到每次都读表的目的,只要 order_flag = true, - // 则强制每次读取 all_sequence_value 一行 + if (true == order_flag && OLD_ACTION == cache_order_mode) { + // When the version is lower than 4.2.3, the cache order mode default cache size is 1 and the + // __all_sequence_value table must be read every time. After the upgrade, this restriction no + // longer exists. ret = cache_size.from(static_cast(1), allocator); // allocator_use_1 } else { cache_size.shadow_copy(option.get_cache_size()); diff --git a/src/share/sequence/ob_sequence_option.h b/src/share/sequence/ob_sequence_option.h index 8e68b1dc5d..5c0b77c4f9 100644 --- a/src/share/sequence/ob_sequence_option.h +++ b/src/share/sequence/ob_sequence_option.h @@ -264,10 +264,6 @@ public: !opt_bitset.has_member(ObSequenceArg::NOORDER)) { order_ = from.order_; } - // The flag field will not be modified through alter, just copy the value in from. - if (OB_SUCC(ret)) { - flag_ = from.flag_; - } return ret; } @@ -279,7 +275,8 @@ public: K_(minvalue), K_(cache), K_(cycle), - K_(order)); + K_(order), + K_(flag)); private: ObSequenceValue increment_by_; diff --git a/src/sql/dtl/ob_dtl_basic_channel.cpp b/src/sql/dtl/ob_dtl_basic_channel.cpp index c1d159e6f4..f0d26731a4 100644 --- a/src/sql/dtl/ob_dtl_basic_channel.cpp +++ b/src/sql/dtl/ob_dtl_basic_channel.cpp @@ -1035,7 +1035,8 @@ int ObDtlBasicChannel::inner_write_msg(const ObDtlMsg &msg, int64_t timeout_ts, } else if (OB_FAIL(msg_writer_->need_new_buffer(msg, eval_ctx, need_size, need_new))) { LOG_WARN("failed to judge need new buffer", K(ret)); } else if (OB_UNLIKELY(need_new)) { - if (OB_FAIL(switch_buffer(need_size, is_eof, timeout_ts))) { + if (OB_FAIL(switch_buffer(need_size, is_eof, timeout_ts, + eval_ctx))) { LOG_WARN("failed to switch buffer", K(ret)); } } @@ -1069,7 +1070,7 @@ int ObDtlBasicChannel::write_msg(const ObDtlMsg &msg, int64_t timeout_ts, } int ObDtlBasicChannel::switch_buffer(const int64_t min_size, const bool is_eof, - const int64_t timeout_ts) + const int64_t timeout_ts, ObEvalCtx *eval_ctx) { int ret = OB_SUCCESS; if (write_buffer_ != nullptr && write_buffer_->pos() != 0) { @@ -1104,6 +1105,28 @@ int ObDtlBasicChannel::switch_buffer(const int64_t min_size, const bool is_eof, write_buffer_->set_sqc_id(dfc_->get_sender_sqc_info().sqc_id_); write_buffer_->set_dfo_id(dfc_->get_sender_sqc_info().dfo_id_); } + if (OB_NOT_NULL(eval_ctx)) { + write_buffer_->set_dop(ObPxSqcUtil::get_actual_worker_count(&eval_ctx->exec_ctx_)); + write_buffer_->set_plan_id(ObPxSqcUtil::get_plan_id(&eval_ctx->exec_ctx_)); + write_buffer_->set_exec_id(ObPxSqcUtil::get_exec_id(&eval_ctx->exec_ctx_)); + write_buffer_->set_session_id(ObPxSqcUtil::get_session_id(&eval_ctx->exec_ctx_)); + ObPhysicalPlanCtx *plan_ctx = eval_ctx->exec_ctx_.get_physical_plan_ctx(); + if (OB_NOT_NULL(plan_ctx) && OB_NOT_NULL(plan_ctx->get_phy_plan())) { + write_buffer_->set_sql_id(plan_ctx->get_phy_plan()->get_sql_id()); + write_buffer_->set_disable_auto_mem_mgr(plan_ctx->get_phy_plan()-> + is_disable_auto_memory_mgr()); + } + ObSQLSessionInfo *sql_session = eval_ctx->exec_ctx_.get_my_session(); + if (OB_NOT_NULL(sql_session)) { + write_buffer_->set_database_id(sql_session->get_database_id()); + } + write_buffer_->set_op_id(get_op_id()); + ObOperatorKit *kit = eval_ctx->exec_ctx_.get_operator_kit(get_op_id()); + if (OB_NOT_NULL(kit) && OB_NOT_NULL(kit->op_)) { + write_buffer_->set_input_rows(kit->op_->get_spec().rows_); + write_buffer_->set_input_width(kit->op_->get_spec().width_); + } + } write_buffer_->timeout_ts() = timeout_ts; msg_writer_->write_msg_type(write_buffer_); write_buffer_->set_data_msg(is_data_msg_); diff --git a/src/sql/dtl/ob_dtl_basic_channel.h b/src/sql/dtl/ob_dtl_basic_channel.h index 7aa3c7c9e7..2cdf46ce21 100644 --- a/src/sql/dtl/ob_dtl_basic_channel.h +++ b/src/sql/dtl/ob_dtl_basic_channel.h @@ -718,7 +718,7 @@ protected: int push_back_send_list(); int wait_unblocking(); int switch_buffer(const int64_t min_size, const bool is_eof, - const int64_t timeout_ts); + const int64_t timeout_ts, ObEvalCtx *eval_ctx); int write_msg(const ObDtlMsg &msg, int64_t timeout_ts, ObEvalCtx *eval_ctx, bool is_eof); int inner_write_msg(const ObDtlMsg &msg, int64_t timeout_ts, ObEvalCtx *eval_ctx, bool is_eof); diff --git a/src/sql/dtl/ob_dtl_interm_result_manager.cpp b/src/sql/dtl/ob_dtl_interm_result_manager.cpp index ec1b8b06f3..d456c09cef 100644 --- a/src/sql/dtl/ob_dtl_interm_result_manager.cpp +++ b/src/sql/dtl/ob_dtl_interm_result_manager.cpp @@ -27,47 +27,14 @@ using namespace common; using namespace sql; using namespace oceanbase::sql::dtl; -void ObDTLIntermResultGC::reset() -{ - expire_keys_.reset(); -} - int ObDTLIntermResultGC::operator() (common::hash::HashMapPair &entry) { int ret = OB_SUCCESS; - int64_t dis = cur_time_ - entry.first.time_us_; - ++interm_cnt_; - if (DUMP == gc_type_) { - if (!entry.second->is_read_ && - OB_SUCCESS == entry.second->ret_ && - cur_time_ - entry.first.start_time_ > DUMP_TIME_THRESHOLD && - dis < 0) { - int64_t dump_time = oceanbase::common::ObTimeUtility::current_time(); - if (OB_FAIL(DTL_IR_STORE_DO(*entry.second, dump, false, true))) { - LOG_WARN("fail to dump interm row store", K(ret)); - } else if (OB_FAIL(DTL_IR_STORE_DO(*entry.second, finish_add_row, true))) { - LOG_WARN("fail to finish add row in interm store", K(ret)); - } else { - LOG_DEBUG("dump interm result success", K(entry.first.channel_id_)); - } - int64_t dump_cost = oceanbase::common::ObTimeUtility::current_time() - dump_time; - entry.second->dump_time_ = 0 == entry.second->dump_time_ ? dump_time : entry.second->dump_time_; - entry.second->dump_cost_ += dump_cost; - } - if (OB_SUCCESS != ret) { - entry.second->ret_ = ret; - // free interm result info datum store in advance for memory optimization. - ObDTLIntermResultManager::free_interm_result_info_store(entry.second); - } - } - - if (CLEAR == gc_type_ && 0 == dump_count_ % 10) { - // 如果已经dump已经做过10次了, 则认为可以清理了. - // 按照设计, dump每隔10秒触发一次, 清理则每隔100秒. - // hard code 超过超时时间30秒. - if (dis > CLEAR_TIME_THRESHOLD) { - ++clean_cnt_; - ret = expire_keys_.push_back(entry.first); + int64_t dis = cur_time_ - entry.first.timeout_ts_; + if (dis > CLEAR_TIME_THRESHOLD) { + ++clean_cnt_; + if (OB_FAIL(expire_keys_.push_back(entry.first))) { + LOG_WARN("clear interm res, but push back failed.", K(ret), K(expire_keys_.count())); } } @@ -87,7 +54,7 @@ void ObAtomicGetIntermResultInfoCall::operator() (common::hash::HashMapPairis_eof_) { entry.second->is_read_ = true; - result_info_guard_.set_result_info(*entry.second); + result_info_guard_.set_result_info(*entry.second, interm_res_manager_); } else { ret_ = OB_HASH_NOT_EXIST; } @@ -97,50 +64,64 @@ void ObAtomicGetIntermResultInfoCall::operator() (common::hash::HashMapPair &entry) { + int ret = OB_SUCCESS; if (OB_NOT_NULL(block_buf_)) { if (OB_SUCCESS != entry.second->ret_) { - ret_ = entry.second->ret_; + ret = entry.second->ret_; } else if (OB_UNLIKELY(!entry.second->is_store_valid())) { - ret_ = OB_INVALID_ARGUMENT; + ret = OB_INVALID_ARGUMENT; + LOG_WARN("store of result info is invalid.", K(ret)); } else { - if (entry.second->use_rich_format_) { - ret_ = entry.second->col_store_->append_block(block_buf_, size_); + int row_cnt_before_append = DTL_IR_STORE_DO(*(entry.second), get_row_cnt_in_memory); + if (OB_FAIL(DTL_IR_STORE_DO_APPEND_BLOCK(*entry.second, block_buf_, size_, true))) { + LOG_WARN("store of result info append_block failed", K(ret)); + } else if (OB_FAIL(interm_res_manager_->process_dump(*(entry.second), mem_profile_info_))) { + LOG_WARN("process_dump failed", K(ret)); } else { - ret_ = entry.second->datum_store_->append_block(block_buf_, size_, true); + int64_t row_cnt_after_append = DTL_IR_STORE_DO(*(entry.second), get_row_cnt_in_memory); + mem_profile_info_->update_row_count(row_cnt_after_append - row_cnt_before_append); } if (is_eof_) { entry.second->is_eof_ = is_eof_; } } } + ret_ = ret; } void ObAtomicAppendPartBlockCall::operator() (common::hash::HashMapPair &entry) { + int ret = OB_SUCCESS; if (OB_NOT_NULL(block_buf_)) { if (OB_SUCCESS != entry.second->ret_) { - ret_ = entry.second->ret_; + ret = entry.second->ret_; } else if (OB_UNLIKELY(!entry.second->is_store_valid())) { - ret_ = OB_INVALID_ARGUMENT; + ret = OB_INVALID_ARGUMENT; + LOG_WARN("store of result info is invalid.", K(ret)); } else { - if (entry.second->use_rich_format_) { - ret_ = entry.second->col_store_->append_block_payload(block_buf_ + start_pos_, length_, rows_); + int64_t row_cnt_before_append = DTL_IR_STORE_DO(*(entry.second), get_row_cnt_in_memory); + if (OB_FAIL(DTL_IR_STORE_DO_APPEND_BLOCK_PAYLOAD(*entry.second, + block_buf_ + start_pos_, length_, rows_, true))) { + LOG_WARN("store of result info append_block_payload failed", K(ret)); + } else if (OB_FAIL(interm_res_manager_->process_dump(*(entry.second), mem_profile_info_))) { + LOG_WARN("process_dump failed", K(ret)); } else { - ret_ = entry.second->datum_store_->append_block_payload(block_buf_ + start_pos_, length_, rows_, true); + int64_t row_cnt_after_append = DTL_IR_STORE_DO(*(entry.second), get_row_cnt_in_memory); + mem_profile_info_->update_row_count(row_cnt_after_append - row_cnt_before_append); } if (is_eof_) { entry.second->is_eof_ = is_eof_; } } } + ret_ = ret; } int ObDTLIntermResultManager::init() { int ret = OB_SUCCESS; uint64_t tenant_id = MTL_ID(); - ObMemAttr attr(tenant_id, "HashBuckDTLINT"); int64_t tenant_mem_limit = lib::get_tenant_memory_limit(tenant_id); double mem_factor = static_cast(tenant_mem_limit) / lib::get_memory_limit(); // less memory for meta tenant @@ -148,11 +129,19 @@ int ObDTLIntermResultManager::init() mem_factor = mem_factor * 0.01; } + ObMemAttr interm_res_hash_buck_attr(tenant_id, "HashBuckDTLINT"); + ObMemAttr mem_profile_hash_buck_attr(tenant_id, "IRMMemHashBuck"); + omt::ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id)); + int64_t cpu_quota_concurrency = tenant_config->cpu_quota_concurrency; if (IS_INIT) { ret = OB_INIT_TWICE; - } else if (OB_FAIL(map_.create(static_cast(DEFAULT_BUCKET_NUM * mem_factor), - attr, attr))) { - LOG_WARN("create hash table failed", K(ret)); + } else if (OB_FAIL(interm_res_map_.create(static_cast(DEFAULT_BUCKET_NUM * mem_factor), + interm_res_hash_buck_attr, interm_res_hash_buck_attr))) { + LOG_WARN("create interm_res hash table failed", K(ret)); + } else if (OB_FAIL(mem_profile_map_.create( + static_cast(MTL_CPU_COUNT() * cpu_quota_concurrency * 2), + mem_profile_hash_buck_attr, mem_profile_hash_buck_attr))) { + LOG_WARN("create mem_profile hash table failed", K(ret)); } else { is_inited_ = true; } @@ -168,12 +157,48 @@ void ObDTLIntermResultManager::destroy() { if (IS_INIT) { erase_tenant_interm_result_info(); - map_.destroy(); + interm_res_map_.destroy(); + // In theory, mem_profile_map_ should be empty at this point. + // This is defensive programming. + clear_mem_profile_map(); // Used to handle scenarios where destroy is called multiple times. is_inited_ = false; } } +int ObDTLIntermResultManager::clear_mem_profile_map() +{ + int ret = OB_SUCCESS; + lib::ObMutexGuard guard(mem_profile_mutex_); + if (!mem_profile_map_.empty()) { + int64_t mem_profile_map_size = mem_profile_map_.size(); + MemProfileMap::bucket_iterator bucket_it = mem_profile_map_.bucket_begin(); + while (bucket_it != mem_profile_map_.bucket_end()) { + while (true) { + ObDTLMemProfileKey key; + { + MemProfileMap::hashtable::bucket_lock_cond blc(*bucket_it); + MemProfileMap::hashtable::readlocker locker(blc.lock()); + MemProfileMap::hashtable::hashbucket::const_iterator node_it = bucket_it->node_begin(); + if (node_it == bucket_it->node_end()) { + break; + } else { + key = node_it->first; + } + } + destroy_mem_profile(key); + } + ++bucket_it; + } + if (OB_SUCC(ret)) { + LOG_INFO("clear_mem_profile_map", K(MTL_ID()), K(mem_profile_map_size)); + } + } + mem_profile_map_.destroy(); + + return ret; +} + void ObDTLIntermResultManager::mtl_destroy(ObDTLIntermResultManager *&dtl_interm_result_manager) { if (nullptr != dtl_interm_result_manager) { @@ -187,7 +212,7 @@ int ObDTLIntermResultManager::get_interm_result_info(ObDTLIntermResultKey &key, { int ret = OB_SUCCESS; ObDTLIntermResultInfo *tmp_result_info = NULL; - if (OB_FAIL(map_.get_refactored(key, tmp_result_info))) { + if (OB_FAIL(interm_res_map_.get_refactored(key, tmp_result_info))) { LOG_TRACE("fail to get row store in result manager", K(ret), K(key.channel_id_)); } else { result_info = *tmp_result_info; @@ -222,8 +247,7 @@ int ObDTLIntermResultManager::create_interm_result_info(ObMemAttr &attr, result_info->is_read_ = false; result_info->trace_id_ = *ObCurTraceId::get_trace_id(); result_info->monitor_info_ = monitor_info; - result_info->tenant_id_ = attr.tenant_id_; - result_info_guard.set_result_info(*result_info); + result_info_guard.set_result_info(*result_info, this); } if (OB_FAIL(ret)) { if (NULL != result_info_buf) { @@ -243,15 +267,25 @@ int ObDTLIntermResultManager::insert_interm_result_info(ObDTLIntermResultKey &ke if (!result_info->is_store_valid()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("row store is null", K(ret)); - } else if (-1 == dir_id_ && OB_FAIL(FILE_MANAGER_INSTANCE_V2.alloc_dir(dir_id_))) { - LOG_WARN("allocate file directory failed", K(ret)); - } else if (FALSE_IT(DTL_IR_STORE_DO(*result_info, set_dir_id, dir_id_))) { - } else if (FALSE_IT(result_info->inc_ref_count())) { - } else if (OB_FAIL(map_.set_refactored(key, result_info))) { - LOG_WARN("fail to get row store in result manager", K(ret)); - dec_interm_result_ref_count(result_info); - } else { - LOG_DEBUG("debug create interm result info", K(key)); + } else if (-1 == dir_id_) { + // The code here is mainly for the use of the temp_table. + // For the px module, + // the dir_id has already been set in the previous access_mem_profile. + if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.alloc_dir(dir_id_))) { + LOG_WARN("allocate file directory failed", K(ret)); + } else { + DTL_IR_STORE_DO(*result_info, set_dir_id, dir_id_); + } + } + + if (OB_SUCC(ret)) { + inc_interm_result_ref_count(result_info); + if (OB_FAIL(interm_res_map_.set_refactored(key, result_info))) { + LOG_WARN("fail to set row store in result manager", K(ret)); + dec_interm_result_ref_count(result_info); + } else { + LOG_DEBUG("debug insert interm result info", K(key)); + } } return ret; } @@ -273,12 +307,33 @@ void ObDTLIntermResultManager::free_interm_result_info_store(ObDTLIntermResultIn } } -void ObDTLIntermResultManager::free_interm_result_info(ObDTLIntermResultInfo *result_info) +int ObDTLIntermResultManager::free_interm_result_info(ObDTLIntermResultInfo *result_info) { + int ret = OB_SUCCESS; if (NULL != result_info) { + // For the px module that accesses automatic memory management, + // the mem_profile_key at this point is valid + // and enters the erase logic of the automatic memory management component. + // For the temp_table module that does not access it, this logic is skipped. + ObDTLMemProfileKey mem_profile_key(result_info->mem_profile_key_); + ObDTLMemProfileInfo *mem_profile_info = nullptr; + if (mem_profile_key.is_valid()) { + if (OB_FAIL(mem_profile_map_.get_refactored(mem_profile_key, + mem_profile_info))) { + LOG_WARN("get mem_profile failed", K(ret), K(mem_profile_key)); + } else if (OB_NOT_NULL(mem_profile_info)) { + mem_profile_info->update_row_count(-DTL_IR_STORE_DO(*result_info, get_row_cnt_in_memory)); + } + } free_interm_result_info_store(result_info); ob_free(result_info); + if (mem_profile_key.is_valid()) { + if (OB_FAIL(dec_mem_profile_ref_count(mem_profile_key, mem_profile_info))) { + LOG_WARN("dec mem_profile ref_count failed", K(ret), K(mem_profile_key)); + } + } } + return ret; } int ObDTLIntermResultManager::erase_interm_result_info(const ObDTLIntermResultKey &key, @@ -286,7 +341,7 @@ int ObDTLIntermResultManager::erase_interm_result_info(const ObDTLIntermResultKe { int ret = OB_SUCCESS; ObDTLIntermResultInfo *result_info = NULL; - if (OB_FAIL(map_.erase_refactored(key, &result_info))) { + if (OB_FAIL(interm_res_map_.erase_refactored(key, &result_info))) { LOG_TRACE("fail to get row store in result manager", K(key), K(ret)); } else { if (need_unregister_check_item_from_dm) { @@ -297,41 +352,28 @@ int ObDTLIntermResultManager::erase_interm_result_info(const ObDTLIntermResultKe return ret; } -int ObDTLIntermResultManager::clear_timeout_result_info(ObDTLIntermResultGC &gc) +int ObDTLIntermResultManager::clear_timeout_result_info() { int ret = OB_SUCCESS; - gc.gc_type_ = ObDTLIntermResultGC::CLEAR; - if (OB_FAIL(map_.foreach_refactored(gc))) { + if (OB_FAIL(interm_res_map_.foreach_refactored(gc_))) { LOG_WARN("fail to get row store in result manager", K(ret)); } else { - if (OB_SUCC(ret)) { - for (int i = 0; i < gc.expire_keys_.count(); ++i) { - ObDTLIntermResultKey &key = gc.expire_keys_.at(i); - if (OB_FAIL(erase_interm_result_info(key))) { - LOG_WARN("fail to erase row store", K(key), K(ret)); - } + for (int i = 0; i < gc_.expire_keys_.count(); ++i) { + ObDTLIntermResultKey &key = gc_.expire_keys_.at(i); + if (OB_FAIL(erase_interm_result_info(key))) { + LOG_WARN("fail to erase row store", K(key), K(ret)); } } } return ret; } -int ObDTLIntermResultManager::dump_result_info(ObDTLIntermResultGC &gc) -{ - int ret = OB_SUCCESS; - gc.gc_type_ = ObDTLIntermResultGC::DUMP; - if (OB_FAIL(map_.foreach_refactored(gc))) { - LOG_WARN("fail to get row store in result manager", K(ret)); - } - return ret; -} - int ObDTLIntermResultManager::atomic_get_interm_result_info(ObDTLIntermResultKey &key, ObDTLIntermResultInfoGuard &guard) { int ret = OB_SUCCESS; - ObAtomicGetIntermResultInfoCall call(guard); - if (OB_FAIL(map_.atomic_refactored(key, call))) { + ObAtomicGetIntermResultInfoCall call(guard, this); + if (OB_FAIL(interm_res_map_.atomic_refactored(key, call))) { LOG_TRACE("fail to get row store in result manager", K(ret)); } else if (OB_SUCCESS != call.ret_) { ret = call.ret_; @@ -340,10 +382,11 @@ int ObDTLIntermResultManager::atomic_get_interm_result_info(ObDTLIntermResultKey return ret; } -int ObDTLIntermResultManager::atomic_append_block(ObDTLIntermResultKey &key, ObAtomicAppendBlockCall &call) +int ObDTLIntermResultManager::atomic_append_block(ObDTLIntermResultKey &key, + ObAtomicAppendBlockCall &call) { int ret = OB_SUCCESS; - if (OB_FAIL(map_.atomic_refactored(key, call))) { + if (OB_FAIL(interm_res_map_.atomic_refactored(key, call))) { LOG_WARN("fail to get row store in result manager", K(ret)); } else if (OB_FAIL(call.ret_)) { LOG_WARN("ObAtomicAppendBlockCall fail", K(ret)); @@ -353,10 +396,11 @@ int ObDTLIntermResultManager::atomic_append_block(ObDTLIntermResultKey &key, ObA return ret; } -int ObDTLIntermResultManager::atomic_append_part_block(ObDTLIntermResultKey &key, ObAtomicAppendPartBlockCall &call) +int ObDTLIntermResultManager::atomic_append_part_block(ObDTLIntermResultKey &key, + ObAtomicAppendPartBlockCall &call) { int ret = OB_SUCCESS; - if (OB_FAIL(map_.atomic_refactored(key, call))) { + if (OB_FAIL(interm_res_map_.atomic_refactored(key, call))) { LOG_WARN("fail to get row store in result manager", K(ret)); } else if (OB_FAIL(call.ret_)) { LOG_WARN("ObAtomicAppendPartBlockCall fail", K(ret)); @@ -366,40 +410,43 @@ int ObDTLIntermResultManager::atomic_append_part_block(ObDTLIntermResultKey &key return ret; } - ObDTLIntermResultManager::ObDTLIntermResultManager() - : map_(), is_inited_(false), dir_id_(-1), gc_() +ObDTLIntermResultManager::ObDTLIntermResultManager() + : interm_res_map_(), is_inited_(false), dir_id_(-1), gc_(), + mem_profile_mutex_(common::ObLatchIds::SQL_MEMORY_MGR_MUTEX_LOCK) { } - ObDTLIntermResultManager::~ObDTLIntermResultManager() +ObDTLIntermResultManager::~ObDTLIntermResultManager() { destroy(); } -int ObDTLIntermResultManager::generate_monitor_info_rows(observer::ObDTLIntermResultMonitorInfoGetter &monitor_info_getter) +int ObDTLIntermResultManager::generate_monitor_info_rows( + observer::ObDTLIntermResultMonitorInfoGetter &monitor_info_getter) { int ret = OB_SUCCESS; - if (OB_FAIL(map_.foreach_refactored(monitor_info_getter))) { - LOG_WARN("fail to generate monitor info array from map", K(ret), K(map_.size())); + if (OB_FAIL(interm_res_map_.foreach_refactored(monitor_info_getter))) { + LOG_WARN("fail to generate monitor info array from map", + K(ret), K(interm_res_map_.size())); if (OB_SIZE_OVERFLOW == ret) { ret = OB_SUCCESS; } } - LOG_TRACE("generate monitor info rows", K(ret), K(map_.size())); + LOG_TRACE("generate monitor info rows", K(ret), K(interm_res_map_.size())); return ret; } int ObDTLIntermResultManager::erase_tenant_interm_result_info() { int ret = OB_SUCCESS; - MAP::bucket_iterator bucket_it = map_.bucket_begin(); - while (bucket_it != map_.bucket_end()) { + IntermResMap::bucket_iterator bucket_it = interm_res_map_.bucket_begin(); + while (bucket_it != interm_res_map_.bucket_end()) { while (true) { ObDTLIntermResultKey key; { - MAP::hashtable::bucket_lock_cond blc(*bucket_it); - MAP::hashtable::readlocker locker(blc.lock()); - MAP::hashtable::hashbucket::const_iterator node_it = bucket_it->node_begin(); + IntermResMap::hashtable::bucket_lock_cond blc(*bucket_it); + IntermResMap::hashtable::readlocker locker(blc.lock()); + IntermResMap::hashtable::hashbucket::const_iterator node_it = bucket_it->node_begin(); if (node_it == bucket_it->node_end()) { break; } else { @@ -411,7 +458,7 @@ int ObDTLIntermResultManager::erase_tenant_interm_result_info() ++bucket_it; } if (OB_SUCC(ret)) { - LOG_INFO("erase_tenant_interm_result_info", K(MTL_ID()), K(map_.size())); + LOG_INFO("erase_tenant_interm_result_info", K(MTL_ID()), K(interm_res_map_.size())); } return ret; } @@ -422,45 +469,41 @@ int ObDTLIntermResultManager::process_interm_result(ObDtlLinkedBuffer *buffer, i if (OB_ISNULL(buffer)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("fail to process buffer", K(ret)); - } else { - // Prevent the tenant from being migrated during the intermediate result transmission process, leading to error 4013. - MAKE_TENANT_SWITCH_SCOPE_GUARD(tenant_guard); - if (OB_FAIL(tenant_guard.switch_to(buffer->tenant_id()))) { - LOG_WARN("switch tenant failed", K(buffer->tenant_id())); - } else if (buffer->is_batch_info_valid()) { - const ObSArray &infos = buffer->get_batch_info(); - for (int64_t i = 0; i < infos.count() && OB_SUCC(ret); i++) { - ObDTLIntermResultKey key; - const ObDtlBatchInfo &batch_info = infos.at(i); - key.time_us_ = buffer->timeout_ts(); - key.batch_id_ = batch_info.batch_id_; - key.channel_id_ = channel_id; - const int64_t start_pos = batch_info.start_; - const int64_t length = batch_info.end_ - start_pos; - const int64_t rows = batch_info.rows_; - const bool is_eof = infos.count() - 1 == i ? buffer->is_eof() : true; - if (OB_FAIL(process_interm_result_inner(*buffer, key, start_pos, length, rows, is_eof, false))) { - LOG_WARN("process interm result inner", K(ret)); - } - } - LOG_TRACE("process interm result", K(buffer->size()), K(buffer->get_batch_info().count()), - K(buffer->get_batch_info())); - } else { + } else if (buffer->is_batch_info_valid()) { + const ObSArray &infos = buffer->get_batch_info(); + for (int64_t i = 0; i < infos.count() && OB_SUCC(ret); ++i) { ObDTLIntermResultKey key; - key.time_us_ = buffer->timeout_ts(); - key.batch_id_ = buffer->get_batch_id(); + const ObDtlBatchInfo &batch_info = infos.at(i); + key.timeout_ts_ = buffer->timeout_ts(); + key.batch_id_ = batch_info.batch_id_; key.channel_id_ = channel_id; - if (OB_FAIL(process_interm_result_inner(*buffer, key, 0, buffer->size(), 0, buffer->is_eof(), true))) { + const int64_t start_pos = batch_info.start_; + const int64_t length = batch_info.end_ - start_pos; + const int64_t rows = batch_info.rows_; + const bool is_eof = infos.count() - 1 == i ? buffer->is_eof() : true; + if (OB_FAIL(process_interm_result_inner(*buffer, key, start_pos, length, rows, is_eof, false))) { LOG_WARN("process interm result inner", K(ret)); } } + LOG_TRACE("process interm result", K(buffer->size()), K(buffer->get_batch_info().count()), + K(buffer->get_batch_info())); + } else { + ObDTLIntermResultKey key; + key.timeout_ts_ = buffer->timeout_ts(); + key.batch_id_ = buffer->get_batch_id(); + key.channel_id_ = channel_id; + if (OB_FAIL(process_interm_result_inner(*buffer, key, 0, buffer->size(), 0, buffer->is_eof(), true))) { + LOG_WARN("process interm result inner", K(ret)); + } + LOG_TRACE("process interm result", K(buffer->size()), K(buffer->get_batch_info().count()), + K(buffer->get_batch_info())); } return ret; } int ObDTLIntermResultManager::process_interm_result_inner(ObDtlLinkedBuffer &buffer, - ObDTLIntermResultKey &key, + ObDTLIntermResultKey &interm_res_key, int64_t start_pos, int64_t length, int64_t rows, @@ -468,13 +511,15 @@ int ObDTLIntermResultManager::process_interm_result_inner(ObDtlLinkedBuffer &buf bool append_whole_block) { int ret = OB_SUCCESS; - ObDTLIntermResultInfo result_info; - if (OB_FAIL(get_interm_result_info(key, result_info))) { + ObDTLIntermResultInfo interm_res_info; + ObDTLIntermResultInfoGuard result_info_guard; + ObDTLMemProfileKey mem_profile_key(buffer.get_px_sequence_id(), buffer.get_dfo_id()); + ObDTLMemProfileInfo *mem_profile_info = NULL; + if (OB_FAIL(get_interm_result_info(interm_res_key, interm_res_info))) { if (OB_HASH_NOT_EXIST == ret) { - ObDTLIntermResultInfoGuard result_info_guard; - ObMemAttr attr(buffer.tenant_id(), "DtlIntermRes", common::ObCtxIds::EXECUTE_CTX_ID); - key.start_time_ = oceanbase::common::ObTimeUtility::current_time(); ret = OB_SUCCESS; + ObMemAttr attr(buffer.tenant_id(), "DtlIntermRes", common::ObCtxIds::EXECUTE_CTX_ID); + interm_res_key.start_time_ = oceanbase::common::ObTimeUtility::current_time(); if (OB_FAIL(create_interm_result_info(attr, result_info_guard, ObDTLIntermResultMonitorInfo(buffer.get_dfo_key().qc_id_, @@ -485,52 +530,64 @@ int ObDTLIntermResultManager::process_interm_result_inner(ObDtlLinkedBuffer &buf LOG_WARN("the store type of interm_res_info is unexpected.", K(result_info_guard.result_info_)); } else if (OB_FAIL(result_info_guard.result_info_->datum_store_->init( 0, buffer.tenant_id(), common::ObCtxIds::EXECUTE_CTX_ID, "DtlIntermRes"))) { - LOG_WARN("fail to init datum_store buffer", K(ret)); - } else if (OB_FAIL(insert_interm_result_info(key, result_info_guard.result_info_))) { + LOG_WARN("fail to init buffer", K(ret)); + } else if (OB_FAIL(access_mem_profile(mem_profile_key, + mem_profile_info, + *result_info_guard.result_info_, + buffer))) { + LOG_WARN("fail to insert access mem_profile", K(ret)); + } else if (OB_FAIL(insert_interm_result_info(interm_res_key, result_info_guard.result_info_))) { LOG_WARN("fail to insert row store", K(ret)); } else { int reg_dm_ret = ObDetectManagerUtils::single_dfo_register_check_item_into_dm( - buffer.get_register_dm_info(), key, result_info_guard.result_info_); + buffer.get_register_dm_info(), interm_res_key, result_info_guard.result_info_); if (OB_SUCCESS != reg_dm_ret) { LOG_WARN("[DM] single dfo fail to register_check_item_into_dm", - K(reg_dm_ret), K(buffer.get_register_dm_info()), K(key)); + K(reg_dm_ret), K(buffer.get_register_dm_info()), K(interm_res_key)); } LOG_TRACE("register_check_item_into_dm", K(reg_dm_ret), - K(buffer.get_register_dm_info()), K(key), + K(buffer.get_register_dm_info()), K(interm_res_key), K(result_info_guard.result_info_->unregister_dm_info_.node_sequence_id_)); } + } else { + LOG_WARN("fail to get interm_result_info", K(ret), K(interm_res_key)); } + } else if (OB_FAIL(mem_profile_map_.get_refactored(mem_profile_key, mem_profile_info))) { + LOG_WARN("The interm result already exists, and fail to get mem_profile_info.", + K(ret), K(interm_res_key), K(mem_profile_key)); } + // append block if (OB_SUCC(ret)) { if (OB_UNLIKELY(length < 0 || start_pos + length > buffer.size())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("out of index", K(ret), K(buffer.size()), K(start_pos), K(length)); } else if (append_whole_block) { - ObAtomicAppendBlockCall call(buffer.buf(), length, is_eof); - if (OB_FAIL(atomic_append_block(key, call))) { + ObAtomicAppendBlockCall call(buffer.buf(), length, is_eof, this, mem_profile_info); + if (OB_FAIL(atomic_append_block(interm_res_key, call))) { if (OB_HASH_NOT_EXIST == ret && - oceanbase::common::ObTimeUtility::current_time() > key.time_us_) { + oceanbase::common::ObTimeUtility::current_time() > interm_res_key.timeout_ts_) { ret = OB_TIMEOUT; } - LOG_WARN("fail to append block", K(ret), K(key.batch_id_)); + LOG_WARN("fail to append block", K(ret), K(interm_res_key.batch_id_)); } else { ret = call.ret_; if (OB_SUCCESS != ret) { - LOG_WARN("fail to append block", K(ret), K(key)); + LOG_WARN("fail to append block", K(ret), K(interm_res_key)); } } } else { - ObAtomicAppendPartBlockCall call(buffer.buf(), start_pos, length, rows, is_eof); - if (OB_FAIL(atomic_append_part_block(key, call))) { + ObAtomicAppendPartBlockCall call(buffer.buf(), start_pos, length, rows, + is_eof, this, mem_profile_info); + if (OB_FAIL(atomic_append_part_block(interm_res_key, call))) { if (OB_HASH_NOT_EXIST == ret && - oceanbase::common::ObTimeUtility::current_time() > key.time_us_) { + oceanbase::common::ObTimeUtility::current_time() > interm_res_key.timeout_ts_) { ret = OB_TIMEOUT; } - LOG_WARN("fail to append block", K(ret), K(key.batch_id_)); + LOG_WARN("fail to append part block", K(ret), K(interm_res_key.batch_id_)); } else { ret = call.ret_; if (OB_SUCCESS != ret) { - LOG_WARN("fail to append block", K(ret), K(key)); + LOG_WARN("fail to append part block", K(ret), K(interm_res_key)); } } } @@ -539,8 +596,8 @@ int ObDTLIntermResultManager::process_interm_result_inner(ObDtlLinkedBuffer &buf if (OB_FAIL(ret)) { // 注意这里理论上也不会有并发问题,因为channel是点对点且串行发送的 // 所以这个接收到了,肯定没有其他线程给这个channel发送 - // 尝试先从hash table中释放(尽早释放内存,其实也可以让timer来清理) - erase_interm_result_info(key); + // 尝试先从hash table中释放(尽早释放内存) + erase_interm_result_info(interm_res_key); } return ret; } @@ -552,18 +609,23 @@ void ObDTLIntermResultManager::inc_interm_result_ref_count(ObDTLIntermResultInfo } } -void ObDTLIntermResultManager::dec_interm_result_ref_count(ObDTLIntermResultInfo *&result_info) +int ObDTLIntermResultManager::dec_interm_result_ref_count(ObDTLIntermResultInfo *&result_info) { + int ret = OB_SUCCESS; if (OB_NOT_NULL(result_info)) { int64_t ref_count = result_info->dec_ref_count(); if (ref_count <= 0) { if (OB_UNLIKELY(ref_count < 0)) { LOG_ERROR_RET(OB_ERR_UNEXPECTED, "ref count of interm result < 0", K(ref_count), KPC(result_info)); } - free_interm_result_info(result_info); - result_info = NULL; + if (OB_FAIL(free_interm_result_info(result_info))) { + LOG_WARN("fail to free interm_result_info", K(ret)); + } else { + result_info = NULL; + } } } + return ret; } void ObDTLIntermResultManager::runTimerTask() @@ -571,54 +633,33 @@ void ObDTLIntermResultManager::runTimerTask() int ret = OB_SUCCESS; gc_.cur_time_ = oceanbase::common::ObTimeUtility::current_time(); gc_.expire_keys_.reset(); - - // Previously, it was placed inside a foreach loop, - // which means that each element would ++gc_.dump_count_. - // In this case, if the cache is a relatively large prime number, - // it may take a long time and it keeps changing, - // so it is unlikely to enter the loop, causing a big bug. - ++gc_.dump_count_; - gc_.interm_cnt_ = 0; - // dump every_10_seconds && not_expired && unused store - if (OB_SUCC(ret)) { - if (OB_FAIL(dump_result_info(gc_))) { - LOG_WARN("fail to for each row store", K(ret)); - } else { - int64_t dump_cost = oceanbase::common::ObTimeUtility::current_time() - gc_.cur_time_; - LOG_INFO("dump dtl interm result cost(us)", K(dump_cost), K(ret), - "interm count", gc_.interm_cnt_, "dump count", gc_.dump_count_); - } - } - gc_.clean_cnt_ = 0; - gc_.interm_cnt_ = 0; - gc_.cur_time_ = oceanbase::common::ObTimeUtility::current_time(); - // Cleaning up expired store - if (OB_SUCC(ret)) { - if (OB_FAIL(clear_timeout_result_info(gc_))) { - LOG_WARN("fail to for each row store", K(ret)); - } else { - int64_t clear_cost = oceanbase::common::ObTimeUtility::current_time() - gc_.cur_time_; - LOG_INFO("clear dtl interm result cost(us)", K(clear_cost), K(ret), - K(gc_.expire_keys_.count()), "dump count", gc_.dump_count_, - "interm count", gc_.interm_cnt_, "clean count", gc_.clean_cnt_); - } + // Cleaning up expired row_store + if (OB_FAIL(clear_timeout_result_info())) { + LOG_WARN("fail to for each row store", K(ret)); + } else { + int64_t clear_cost = oceanbase::common::ObTimeUtility::current_time() - gc_.cur_time_; + LOG_INFO("clear dtl interm result cost(us)", K(clear_cost), K(ret), + K(gc_.expire_keys_.count()), "dump count", gc_.dump_count_, + "clean count", gc_.clean_cnt_); } } -void ObDTLIntermResultInfoGuard::set_result_info(ObDTLIntermResultInfo &result_info) +void ObDTLIntermResultInfoGuard::set_result_info(ObDTLIntermResultInfo &result_info, + ObDTLIntermResultManager *interm_res_manager) { if (&result_info != result_info_) { reset(); ObDTLIntermResultManager::inc_interm_result_ref_count(&result_info); result_info_ = &result_info; + interm_res_manager_ = interm_res_manager; } } void ObDTLIntermResultInfoGuard::reset() { if (NULL != result_info_) { - ObDTLIntermResultManager::dec_interm_result_ref_count(result_info_); + interm_res_manager_->dec_interm_result_ref_count(result_info_); result_info_ = NULL; } } @@ -659,3 +700,211 @@ void ObDTLIntermResultGCTask::runTimerTask() { dtl_interm_result_manager_->runTimerTask(); } + +int ObDTLIntermResultManager::process_dump(ObDTLIntermResultInfo &result_info, + ObDTLMemProfileInfo *mem_profile_info) +{ + int ret = OB_SUCCESS; + bool updated = false; + bool dumped = false; + if (OB_ISNULL(mem_profile_info)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("mem_profile_info is null", K(ret), K(result_info.mem_profile_key_)); + } else { + { + lib::ObMutexGuard guard(mem_profile_info->mutex_); + ObDTLIntermResultCheckUpdateMem check_update_mem(mem_profile_info->row_count_); + ObDTLIntermResultCheckDump check_dump(mem_profile_info->sql_mem_processor_.get_data_size()); + if (OB_FAIL(mem_profile_info->sql_mem_processor_.update_max_available_mem_size_periodically( + &mem_profile_info->allocator_, + check_update_mem, + updated))) { + LOG_WARN("failed to update max available memory size periodically", K(ret)); + } else if (need_dump(mem_profile_info) && GCONF.is_sql_operator_dump_enabled() + && OB_FAIL(mem_profile_info->sql_mem_processor_.extend_max_memory_size( + &mem_profile_info->allocator_, + check_dump, + dumped, mem_profile_info->sql_mem_processor_.get_data_size()))) { + LOG_WARN("failed to extend max memory size", K(ret)); + } + } + // dump logic + if (OB_SUCC(ret) && dumped) { + // For the sake of performance, this log section does not have lock protection, + // so the information printed may not be accurate. + LOG_DEBUG("begin dump dtl interm result", K(ret), + K(mem_profile_info->sql_mem_processor_.get_data_size()), + K(mem_profile_info->sql_mem_processor_.get_mem_bound())); + int64_t dump_begin_time = oceanbase::common::ObTimeUtility::current_time(); + ++gc_.dump_count_; + if (OB_FAIL(DTL_IR_STORE_DO_DUMP(result_info, false, true))) { + LOG_WARN("fail to dump interm row store", K(ret)); + // Used to forcefully write buffer data to disk + // to prevent errors of not being able to read the remaining data. + } else if (OB_FAIL(DTL_IR_STORE_DO(result_info, finish_add_row, true))) { + LOG_WARN("fail to finish add row in interm store", K(ret)); + } else { + mem_profile_info->set_number_pass(1); + int64_t dump_cost = oceanbase::common::ObTimeUtility::current_time() - dump_begin_time; + LOG_DEBUG("dump dtl interm result cost(us)", K(dump_cost), K(ret), + "dump count", gc_.dump_count_, + "mem profile", result_info.mem_profile_key_, + K(mem_profile_info->sql_mem_processor_.get_data_size()), + K(mem_profile_info->sql_mem_processor_.get_mem_bound())); + } + } + } + + return ret; +} + +// Currently, only interm results of parallel execution (px) +// are integrated with automatic memory management. +int ObDTLIntermResultManager::access_mem_profile(const ObDTLMemProfileKey &mem_profile_key, + ObDTLMemProfileInfo *&mem_profile_info, + ObDTLIntermResultInfo &interm_res_info, + ObDtlLinkedBuffer &buffer) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(mem_profile_map_.get_refactored(mem_profile_key, mem_profile_info))) { + if (ret == OB_HASH_NOT_EXIST) { + ret = OB_SUCCESS; + if (OB_FAIL(init_mem_profile(mem_profile_key, mem_profile_info, buffer))) { + LOG_WARN("fail to init mem_profile", K(ret), K(mem_profile_key)); + } + } else { + LOG_WARN("fail to get mem_profile", K(ret), K(mem_profile_key)); + } + } + if (OB_SUCC(ret) && OB_NOT_NULL(mem_profile_info)) { + inc_mem_profile_ref_count(mem_profile_info); + DTL_IR_STORE_DO(interm_res_info, set_allocator, mem_profile_info->allocator_); + DTL_IR_STORE_DO(interm_res_info, set_callback, mem_profile_info); + DTL_IR_STORE_DO(interm_res_info, set_dir_id, mem_profile_info->sql_mem_processor_.get_dir_id()); + interm_res_info.mem_profile_key_ = mem_profile_key; + LOG_DEBUG("Current situation of accessing intermediate results in the profile.", + K(mem_profile_info->ref_count_), K(mem_profile_key), K(buffer)); + } + return ret; +} + +int ObDTLIntermResultManager::init_mem_profile(const ObDTLMemProfileKey &key, + ObDTLMemProfileInfo *&info, + ObDtlLinkedBuffer &buffer) +{ + int ret = OB_SUCCESS; + if (buffer.seq_no() > 1) { // seq_no begin from 1 + ret = OB_INVALID_ARGUMENT; + LOG_WARN("The buffer is not the first packet, \ + but the corresponding mem_profile does not exist.", + K(buffer.seq_no())); + } + lib::ObMutexGuard guard(mem_profile_mutex_); + // Possible scenario: Multiple interm results accessing the same mem_profile. + // Through lock control, when the first one is initialized, the second one tries to initialize again. + // At this point, by calling get_refactored, + // it detects that the other end has already initialized the current mem_profile, + // and it directly exits. + if (OB_FAIL(ret)) { + } else if (OB_FAIL(mem_profile_map_.get_refactored(key, info))) { + if (ret == OB_HASH_NOT_EXIST) { + ret = OB_SUCCESS; + void *info_buf = nullptr; + ObMemAttr mem_info_attr(MTL_ID(), "IRMMemInfo", common::ObCtxIds::EXECUTE_CTX_ID); + ObMemAttr allocator_attr(MTL_ID(), "DtlIntermRes", common::ObCtxIds::WORK_AREA); + int64_t cache_size = buffer.get_input_rows() * buffer.get_input_width(); + if (cache_size <= 0 || cache_size > ObDTLMemProfileInfo::CACHE_SIZE) { + cache_size = ObDTLMemProfileInfo::CACHE_SIZE; + } + if (OB_ISNULL(info_buf = ob_malloc(sizeof(ObDTLMemProfileInfo), mem_info_attr))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("fail to alloc mem_profile_info", K(ret)); + } else { + info = new(info_buf) ObDTLMemProfileInfo(MTL_ID()); + if (OB_FAIL(info->allocator_.init( + lib::ObMallocAllocator::get_instance(), + OB_MALLOC_NORMAL_BLOCK_SIZE, + allocator_attr))) { + LOG_WARN("failed to init fifo allocator", K(ret)); + info->allocator_.reset(); + ob_free(info); + info = NULL; + } else if (OB_FAIL(info->sql_mem_processor_.init( + &info->allocator_, + MTL_ID(), + cache_size, + PHY_PX_FIFO_RECEIVE, + buffer.get_op_id(), + &buffer))) { + LOG_WARN("failed to init sql memory manager processor", K(ret)); + } else if (OB_FAIL(mem_profile_map_.set_refactored(key, info))) { + LOG_WARN("fail to set row store in result manager", K(ret)); + } + if (OB_FAIL(ret)) { + free_mem_profile(info); + } + } + } else { + LOG_WARN("fail to get mem_profile", K(ret), K(key)); + } + } + return ret; +} + +int ObDTLIntermResultManager::destroy_mem_profile(const ObDTLMemProfileKey &key) +{ + int ret = OB_SUCCESS; + ObDTLMemProfileInfo *info = nullptr; + if (OB_FAIL(mem_profile_map_.erase_refactored(key, &info))) { + // The reason for the nonexistence of the corresponding mem_profile is as follows: + // 1. The ref_cnt of the current thread is decremented (which becomes 1). + // 2. The ref_cnt of another thread is decremented, and it executes destroy. + // 3. The current thread checks the ref_cnt (which is now 0) to execute destroy, + // but it finds that the mem_profile has already been released at this time. + if (ret == OB_HASH_NOT_EXIST) { + ret = OB_SUCCESS; + } else { + LOG_WARN("erase mem_profile failed", K(ret), K(key)); + } + } else { + free_mem_profile(info); + } + return ret; +} + +void ObDTLIntermResultManager::inc_mem_profile_ref_count(ObDTLMemProfileInfo *info) +{ + if (OB_NOT_NULL(info)) { + ATOMIC_INC(&info->ref_count_); + } +} + +int ObDTLIntermResultManager::dec_mem_profile_ref_count(const ObDTLMemProfileKey &key, + ObDTLMemProfileInfo *&info) +{ + int ret = OB_SUCCESS; + if (OB_NOT_NULL(info)) { + int64_t ref_count = ATOMIC_SAF(&info->ref_count_, 1); + if (ref_count <= 0) { + if (OB_UNLIKELY(ref_count < 0)) { + LOG_ERROR_RET(OB_ERR_UNEXPECTED, "ref count of mem_profile < 0", K(ref_count), K(key)); + } + lib::ObMutexGuard guard(mem_profile_mutex_); + if (OB_FAIL(destroy_mem_profile(key))) { + LOG_WARN("destroy mem_profile failed!", K(ret), K(key)); + } + } + } + return ret; +} + +void ObDTLIntermResultManager::free_mem_profile(ObDTLMemProfileInfo *&info) +{ + int ret = OB_SUCCESS; + if (OB_NOT_NULL(info)) { + info->sql_mem_processor_.unregister_profile(); + info->allocator_.reset(); + ob_free(info); + info = NULL; + } +} diff --git a/src/sql/dtl/ob_dtl_interm_result_manager.h b/src/sql/dtl/ob_dtl_interm_result_manager.h index 11e855f88e..859813a962 100644 --- a/src/sql/dtl/ob_dtl_interm_result_manager.h +++ b/src/sql/dtl/ob_dtl_interm_result_manager.h @@ -22,6 +22,8 @@ #include "lib/allocator/ob_allocator.h" #include "share/detect/ob_detectable_id.h" #include "sql/engine/basic/ob_temp_column_store.h" +#include "sql/engine/ob_tenant_sql_memory_manager.h" +#include "src/sql/engine/ob_sql_mem_mgr_processor.h" namespace oceanbase { @@ -49,12 +51,104 @@ struct ObDTLIntermResultMonitorInfo }; class ObDtlLinkedBuffer; + +struct ObDTLMemProfileKey { + ObDTLMemProfileKey(uint64_t px_sequence_id, int64_t dfo_id) + : px_sequence_id_(px_sequence_id), dfo_id_(dfo_id) {} + + ObDTLMemProfileKey() + : px_sequence_id_(-1), dfo_id_(-1) {} + + explicit ObDTLMemProfileKey(const ObDTLMemProfileKey &key) + : px_sequence_id_(key.px_sequence_id_), dfo_id_(key.dfo_id_) {} + + inline uint64_t hash() const + { + uint64_t hash_val = 0; + hash_val = common::murmurhash(&px_sequence_id_, sizeof(uint64_t), 0); + hash_val = common::murmurhash(&dfo_id_, sizeof(int64_t), hash_val); + return hash_val; + } + int hash(uint64_t &hash_val) const { hash_val = hash(); return OB_SUCCESS; } + + inline bool operator==(const ObDTLMemProfileKey& key) const + { + return px_sequence_id_ == key.px_sequence_id_ && dfo_id_ == key.dfo_id_; + } + + inline bool is_valid() { + return ((px_sequence_id_ >= 0) && (dfo_id_ >= 0)); + } + + TO_STRING_KV(K(px_sequence_id_), K(dfo_id_)); + + uint64_t px_sequence_id_; + int64_t dfo_id_; +}; + +class ObDTLMemProfileInfo : public ObSqlMemoryCallback +{ +public: + ObDTLMemProfileInfo(const uint64_t tenant_id) + : allocator_(tenant_id), + profile_(ObSqlWorkAreaType::HASH_WORK_AREA), + sql_mem_processor_(profile_), + ref_count_(0), row_count_(0), + mutex_(common::ObLatchIds::SQL_MEMORY_MGR_MUTEX_LOCK) {} + ~ObDTLMemProfileInfo() {} + + // The local channel and the rpc channel may modify the interm results concurrently, + // and these interme results may be linked to the same profile. + // Therefore, access to the profile needs to be protected by locks + // to prevent concurrent modification issues. + void alloc(int64_t size) + { + lib::ObMutexGuard guard(mutex_); + sql_mem_processor_.alloc(size); + } + + void free(int64_t size) + { + lib::ObMutexGuard guard(mutex_); + sql_mem_processor_.free(size); + } + + void dumped(int64_t size) + { + lib::ObMutexGuard guard(mutex_); + sql_mem_processor_.dumped(size); + } + + void set_number_pass(int32_t num_pass) + { + lib::ObMutexGuard guard(mutex_); + sql_mem_processor_.set_number_pass(num_pass); + } + + void update_row_count(int64_t size) { + lib::ObMutexGuard guard(mutex_); + row_count_ += size; + } + + common::ObFIFOAllocator allocator_; + ObSqlWorkAreaProfile profile_; + ObSqlMemMgrProcessor sql_mem_processor_; + + int64_t ref_count_; + int64_t row_count_; + + static const int64_t CACHE_SIZE = 16 * 1024 * 1024; // 16M + lib::ObMutex mutex_; + + TO_STRING_KV(K(ref_count_), K(row_count_)); +}; + struct ObDTLIntermResultKey { - ObDTLIntermResultKey() : channel_id_(0), time_us_(0), + ObDTLIntermResultKey() : channel_id_(0), timeout_ts_(0), start_time_(0), batch_id_(0) {} int64_t channel_id_; - int64_t time_us_; + int64_t timeout_ts_; int64_t start_time_; int64_t batch_id_; inline uint64_t hash() const @@ -66,11 +160,11 @@ struct ObDTLIntermResultKey } int hash(uint64_t &hash_val) const { hash_val = hash(); return OB_SUCCESS; } - inline bool operator==(const ObDTLIntermResultKey& key) const + inline bool operator==(const ObDTLIntermResultKey &key) const { return channel_id_ == key.channel_id_ && batch_id_ == key.batch_id_; } - TO_STRING_KV(K(channel_id_), K(batch_id_), K(time_us_), K(start_time_)); + TO_STRING_KV(K(channel_id_), K(batch_id_), K(timeout_ts_), K(start_time_)); }; struct ObDTLIntermResultInfo @@ -79,7 +173,8 @@ struct ObDTLIntermResultInfo ObDTLIntermResultInfo() : datum_store_(NULL), col_store_(NULL), ret_(common::OB_SUCCESS), is_read_(false), is_eof_(false), ref_count_(0), - trace_id_(), dump_time_(0), dump_cost_(0), unregister_dm_info_(), use_rich_format_(false) + trace_id_(), dump_time_(0), dump_cost_(0), unregister_dm_info_(), + use_rich_format_(false), mem_profile_key_() {} ~ObDTLIntermResultInfo() {} bool is_store_valid() const { return use_rich_format_ ? NULL != col_store_ : NULL != datum_store_; } @@ -100,7 +195,8 @@ public: K_(ref_count), K_(dump_cost), K_(monitor_info), - K_(use_rich_format) + K_(use_rich_format), + K_(mem_profile_key) ); sql::ObChunkDatumStore *datum_store_; @@ -114,75 +210,109 @@ public: int64_t dump_cost_; common::ObUnregisterDmInfo unregister_dm_info_; ObDTLIntermResultMonitorInfo monitor_info_; - uint64_t tenant_id_; bool use_rich_format_; + ObDTLMemProfileKey mem_profile_key_; }; struct ObDTLIntermResultInfoGuard { - ObDTLIntermResultInfoGuard() : result_info_(NULL) {} +public: + ObDTLIntermResultInfoGuard() : result_info_(NULL), interm_res_manager_(NULL) {} ~ObDTLIntermResultInfoGuard() { reset(); } - void set_result_info(ObDTLIntermResultInfo &result_info); + void set_result_info(ObDTLIntermResultInfo &result_info, + ObDTLIntermResultManager *interm_res_manager); void reset(); ObDTLIntermResultInfo *result_info_; +private: + ObDTLIntermResultManager *interm_res_manager_; }; // helper macro to dispatch action to datum_store_ / col_store_ #define DTL_IR_STORE_DO(ir, act, ...) \ ((ir).use_rich_format_ ? ((ir).col_store_->act(__VA_ARGS__)) : ((ir).datum_store_->act(__VA_ARGS__))) +#define DTL_IR_STORE_DO_APPEND_BLOCK(ir, buf, size, need_swizzling) \ + ((ir).use_rich_format_ ? ((ir).col_store_->append_block(buf, size)) : \ + ((ir).datum_store_->append_block(buf, size, need_swizzling))) + +#define DTL_IR_STORE_DO_APPEND_BLOCK_PAYLOAD(ir, payload, size, rows, need_swizzling) \ + ((ir).use_rich_format_ ? ((ir).col_store_->append_block_payload(payload, size, rows)) : \ + ((ir).datum_store_->append_block_payload(payload, size, rows, need_swizzling))) + +#define DTL_IR_STORE_DO_DUMP(ir, reuse, all_dump) \ + ((ir).use_rich_format_ ? ((ir).col_store_->dump(all_dump)) : \ + ((ir).datum_store_->dump(reuse, all_dump))) + class ObDTLIntermResultGC { friend class ObDTLIntermResultManager; public: ObDTLIntermResultGC() - : cur_time_(0), expire_keys_(), gc_type_(NOT_INIT), dump_count_(0), - interm_cnt_(0), clean_cnt_(0) - {} + : cur_time_(0), expire_keys_(), + dump_count_(0), clean_cnt_(0) {} virtual ~ObDTLIntermResultGC() = default; - void reset(); int operator() (common::hash::HashMapPair &entry); -public: - const static int64_t DUMP_TIME_THRESHOLD = 10 * 1000L * 1000L; // 超过10秒dump - const static int64_t CLEAR_TIME_THRESHOLD = 10 * 1000L * 1000L; // 超过10秒清理 -public: - enum ObGCType { - NOT_INIT = 0, - CLEAR = 1, - DUMP = 2 - }; private: DISALLOW_COPY_AND_ASSIGN(ObDTLIntermResultGC); + +public: + const static int64_t CLEAR_TIME_THRESHOLD = 10 * 1000L * 1000L; // 超过10秒清理 private: int64_t cur_time_; common::ObSEArray expire_keys_; - ObGCType gc_type_; int64_t dump_count_; - int64_t interm_cnt_; int64_t clean_cnt_; }; +class ObDTLIntermResultCheckUpdateMem +{ +public: + explicit ObDTLIntermResultCheckUpdateMem(int64_t cur_row_count) : cur_row_count_(cur_row_count) {} + bool operator()(int64_t max_row_count) { + return cur_row_count_ > max_row_count; + } +private: + int64_t cur_row_count_; +}; + +class ObDTLIntermResultCheckDump +{ +public: + explicit ObDTLIntermResultCheckDump(int64_t cur_mem_size) : cur_mem_size_(cur_mem_size) {} + bool operator()(int64_t max_memory_size) { + return cur_mem_size_ > max_memory_size; + } +private: + int64_t cur_mem_size_; +}; + class ObAtomicGetIntermResultInfoCall { public: - ObAtomicGetIntermResultInfoCall(ObDTLIntermResultInfoGuard &guard) : - result_info_guard_(guard), ret_(OB_SUCCESS) {} + explicit ObAtomicGetIntermResultInfoCall(ObDTLIntermResultInfoGuard &guard, + ObDTLIntermResultManager *interm_res_manager) : + result_info_guard_(guard), interm_res_manager_(interm_res_manager), ret_(OB_SUCCESS) {} ~ObAtomicGetIntermResultInfoCall() = default; void operator() (common::hash::HashMapPair &entry); public: ObDTLIntermResultInfoGuard &result_info_guard_; + ObDTLIntermResultManager *interm_res_manager_; int ret_; }; class ObAtomicAppendBlockCall { public: - ObAtomicAppendBlockCall(char *buf, int64_t size, bool is_eof) : block_buf_(buf), - size_(size), ret_(common::OB_SUCCESS), is_eof_(is_eof) {} + ObAtomicAppendBlockCall(char *buf, int64_t size, bool is_eof, + ObDTLIntermResultManager *interm_res_manager, + ObDTLMemProfileInfo *mem_profile_info) + : block_buf_(buf), size_(size), ret_(common::OB_SUCCESS), is_eof_(is_eof), + interm_res_manager_(interm_res_manager), + mem_profile_info_(mem_profile_info) {} ~ObAtomicAppendBlockCall() = default; void operator() (common::hash::HashMapPair &entry); @@ -192,14 +322,21 @@ public: int64_t size_; int ret_; bool is_eof_; + ObDTLIntermResultManager *interm_res_manager_; + ObDTLMemProfileInfo *mem_profile_info_; }; class ObAtomicAppendPartBlockCall { public: - ObAtomicAppendPartBlockCall(char *buf, int64_t start_pos, int64_t len, int64_t rows, bool is_eof) + ObAtomicAppendPartBlockCall(char *buf, int64_t start_pos, int64_t len, + int64_t rows, bool is_eof, + ObDTLIntermResultManager *interm_res_manager, + ObDTLMemProfileInfo *mem_profile_info) : block_buf_(buf), start_pos_(start_pos), length_(len), rows_(rows), - ret_(common::OB_SUCCESS), is_eof_(is_eof) {} + ret_(common::OB_SUCCESS), is_eof_(is_eof), + interm_res_manager_(interm_res_manager), + mem_profile_info_(mem_profile_info) {} ~ObAtomicAppendPartBlockCall() = default; void operator() (common::hash::HashMapPair &entry); @@ -210,6 +347,8 @@ public: int64_t rows_; int ret_; bool is_eof_; + ObDTLIntermResultManager *interm_res_manager_; + ObDTLMemProfileInfo *mem_profile_info_; }; class ObDTLIntermResultGCTask : public common::ObTimerTask @@ -226,6 +365,9 @@ public: class ObDTLIntermResultManager { friend class ObDTLIntermResultGCTask; + friend class ObAtomicAppendBlockCall; + friend class ObAtomicAppendPartBlockCall; + friend struct ObDTLIntermResultInfoGuard; public: int process_interm_result(ObDtlLinkedBuffer *buffer, int64_t channel_id); int process_interm_result_inner(ObDtlLinkedBuffer &buffer, @@ -235,15 +377,13 @@ public: int64_t rows, bool is_eof, bool append_whole_block); - typedef common::hash::ObHashMap MAP; int get_interm_result_info(ObDTLIntermResultKey &key, ObDTLIntermResultInfo &result_info); int create_interm_result_info(ObMemAttr &attr, ObDTLIntermResultInfoGuard &result_info_guard, const ObDTLIntermResultMonitorInfo &monitor_info, bool use_rich_format = false); int erase_interm_result_info(const ObDTLIntermResultKey &key, bool need_unregister_check_item_from_dm=true); int insert_interm_result_info(ObDTLIntermResultKey &key, ObDTLIntermResultInfo *&result_info); // 以下两个接口会持有bucket读锁. - int clear_timeout_result_info(ObDTLIntermResultGC &gc); - int dump_result_info(ObDTLIntermResultGC &gc); + int clear_timeout_result_info(); // atomic_get_interm_result_info接口. // 将会持写锁标记result info为已读. // 后台的dump线程遇到已读的row_store将不会dump. @@ -257,9 +397,9 @@ public: int generate_monitor_info_rows(observer::ObDTLIntermResultMonitorInfoGetter &monitor_info_getter); int erase_tenant_interm_result_info(); static void free_interm_result_info_store(ObDTLIntermResultInfo *result_info); - static void free_interm_result_info(ObDTLIntermResultInfo *result_info); + int free_interm_result_info(ObDTLIntermResultInfo *result_info); static void inc_interm_result_ref_count(ObDTLIntermResultInfo *result_info); - static void dec_interm_result_ref_count(ObDTLIntermResultInfo *&result_info); + int dec_interm_result_ref_count(ObDTLIntermResultInfo *&result_info); void runTimerTask(); static int mtl_start(ObDTLIntermResultManager *&dtl_interm_result_manager); static void mtl_stop(ObDTLIntermResultManager *&dtl_interm_result_manager); @@ -268,16 +408,47 @@ public: ObDTLIntermResultManager(); ~ObDTLIntermResultManager(); + +private: + int access_mem_profile(const ObDTLMemProfileKey &mem_profile_key, + ObDTLMemProfileInfo *&mem_profile_info, + ObDTLIntermResultInfo &interm_res_info, + ObDtlLinkedBuffer &buffer); + int init_mem_profile(const ObDTLMemProfileKey& key, + ObDTLMemProfileInfo *&info, + ObDtlLinkedBuffer &buffer); + int destroy_mem_profile(const ObDTLMemProfileKey& key); + + void inc_mem_profile_ref_count(ObDTLMemProfileInfo *info); + int dec_mem_profile_ref_count(const ObDTLMemProfileKey &key, + ObDTLMemProfileInfo *&info); + void free_mem_profile(ObDTLMemProfileInfo *&info); + + int process_dump(ObDTLIntermResultInfo &result_info, + ObDTLMemProfileInfo *mem_profile_info); + bool need_dump(ObDTLMemProfileInfo *mem_profile_info) + { return mem_profile_info->sql_mem_processor_.get_data_size() > + mem_profile_info->sql_mem_processor_.get_mem_bound(); } + int clear_mem_profile_map(); + private: // 由于此中间结果管理器是全局结构, 基于性能考虑, 减少锁冲突设置bucket_num为50w. static const int64_t DEFAULT_BUCKET_NUM = 500000; //50w - static const int64_t MAX_TENANT_MEM_LIMIT = 17179869184; //16G + private: - MAP map_; + typedef common::hash::ObHashMap IntermResMap; + typedef common::hash::ObHashMap MemProfileMap; + typedef IntermResMap::iterator IntermResMapIter; + typedef MemProfileMap::iterator MemProfileMapIter; +private: + IntermResMap interm_res_map_; + MemProfileMap mem_profile_map_; bool is_inited_; int64_t dir_id_; ObDTLIntermResultGC gc_; ObDTLIntermResultGCTask gc_task_; + lib::ObMutex mem_profile_mutex_; + private: DISALLOW_COPY_AND_ASSIGN(ObDTLIntermResultManager); }; diff --git a/src/sql/dtl/ob_dtl_linked_buffer.cpp b/src/sql/dtl/ob_dtl_linked_buffer.cpp index 0626f37c33..b48d93c551 100644 --- a/src/sql/dtl/ob_dtl_linked_buffer.cpp +++ b/src/sql/dtl/ob_dtl_linked_buffer.cpp @@ -27,6 +27,44 @@ OB_SERIALIZE_MEMBER(ObDtlDfoKey, server_id_, px_sequence_id_, qc_id_, dfo_id_); OB_SERIALIZE_MEMBER(ObDtlBatchInfo, batch_id_, start_, end_, rows_); +OB_DEF_SERIALIZE(ObDtlOpInfo) +{ + using namespace oceanbase::common; + int ret = OB_SUCCESS; + LST_DO_CODE(OB_UNIS_ENCODE, dop_, plan_id_, exec_id_, session_id_, database_id_); + if (OB_SUCC(ret)) { + MEMCPY(buf + pos, sql_id_, common::OB_MAX_SQL_ID_LENGTH + 1); + pos += common::OB_MAX_SQL_ID_LENGTH + 1; + } + LST_DO_CODE(OB_UNIS_ENCODE, op_id_, input_rows_, input_width_, + disable_auto_mem_mgr_); + return ret; +} + +OB_DEF_DESERIALIZE(ObDtlOpInfo) +{ + using namespace oceanbase::common; + int ret = OB_SUCCESS; + LST_DO_CODE(OB_UNIS_DECODE, dop_, plan_id_, exec_id_, session_id_, database_id_); + if (OB_SUCC(ret)) { + MEMCPY(sql_id_, (char*)buf + pos, common::OB_MAX_SQL_ID_LENGTH + 1); + pos += common::OB_MAX_SQL_ID_LENGTH + 1; + } + LST_DO_CODE(OB_UNIS_DECODE, op_id_, input_rows_, input_width_, + disable_auto_mem_mgr_); + return ret; +} + +OB_DEF_SERIALIZE_SIZE(ObDtlOpInfo) +{ + int64_t len = 0; + LST_DO_CODE(OB_UNIS_ADD_LEN, dop_, plan_id_, exec_id_, session_id_, database_id_); + len += common::OB_MAX_SQL_ID_LENGTH + 1; + LST_DO_CODE(OB_UNIS_ADD_LEN, op_id_, input_rows_, input_width_, + disable_auto_mem_mgr_); + return len; +} + int ObDtlLinkedBuffer::deserialize_msg_header(const ObDtlLinkedBuffer &buffer, ObDtlMsgHeader &header, bool keep_pos /*= false*/) @@ -138,6 +176,9 @@ OB_DEF_SERIALIZE(ObDtlLinkedBuffer) if (OB_SUCC(ret)) { LST_DO_CODE(OB_UNIS_ENCODE, row_meta_); } + if (OB_SUCC(ret) && seq_no_ == 1) { + LST_DO_CODE(OB_UNIS_ENCODE, op_info_); + } } } return ret; @@ -179,6 +220,9 @@ OB_DEF_DESERIALIZE(ObDtlLinkedBuffer) if (OB_SUCC(ret)) { LST_DO_CODE(OB_UNIS_DECODE, row_meta_); } + if (OB_SUCC(ret) && seq_no_ == 1) { + LST_DO_CODE(OB_UNIS_DECODE, op_info_); + } } if (OB_SUCC(ret)) { (void)ObSQLUtils::adjust_time_by_ntp_offset(timeout_ts_); @@ -216,13 +260,16 @@ OB_DEF_SERIALIZE_SIZE(ObDtlLinkedBuffer) use_interm_result_, batch_id_, batch_info_valid_); - if (batch_info_valid_) { - LST_DO_CODE(OB_UNIS_ADD_LEN, batch_info_); - } - LST_DO_CODE(OB_UNIS_ADD_LEN, dfo_id_, sqc_id_); - LST_DO_CODE(OB_UNIS_ADD_LEN, enable_channel_sync_); - LST_DO_CODE(OB_UNIS_ADD_LEN, register_dm_info_); - LST_DO_CODE(OB_UNIS_ADD_LEN, row_meta_); + if (batch_info_valid_) { + LST_DO_CODE(OB_UNIS_ADD_LEN, batch_info_); + } + LST_DO_CODE(OB_UNIS_ADD_LEN, dfo_id_, sqc_id_); + LST_DO_CODE(OB_UNIS_ADD_LEN, enable_channel_sync_); + LST_DO_CODE(OB_UNIS_ADD_LEN, register_dm_info_); + LST_DO_CODE(OB_UNIS_ADD_LEN, row_meta_); + if (seq_no_ == 1) { + LST_DO_CODE(OB_UNIS_ADD_LEN, op_info_); + } return len; } diff --git a/src/sql/dtl/ob_dtl_linked_buffer.h b/src/sql/dtl/ob_dtl_linked_buffer.h index c8646145de..80c0bd663c 100644 --- a/src/sql/dtl/ob_dtl_linked_buffer.h +++ b/src/sql/dtl/ob_dtl_linked_buffer.h @@ -78,6 +78,96 @@ public: int64_t dfo_id_; }; +class ObDtlOpInfo +{ + OB_UNIS_VERSION(1); +public: + ObDtlOpInfo() : + dop_(-1), plan_id_(-1), exec_id_(-1), session_id_(-1), database_id_(0), + op_id_(UINT64_MAX), input_rows_(0), input_width_(-1), + disable_auto_mem_mgr_(false) + { + sql_id_[0] = '\0'; + } + uint64_t hash() const + { + uint64_t val = common::murmurhash(&dop_, sizeof(dop_), 0); + val = common::murmurhash(&plan_id_, sizeof(plan_id_), val); + val = common::murmurhash(&exec_id_, sizeof(exec_id_), val); + val = common::murmurhash(&session_id_, sizeof(session_id_), val); + val = common::murmurhash(&database_id_, sizeof(database_id_), val); + val = common::murmurhash(sql_id_, common::OB_MAX_SQL_ID_LENGTH + 1, val); + return val; + } + int hash(uint64_t &hash_val) const { hash_val = hash(); return OB_SUCCESS; } + + bool operator== (const ObDtlOpInfo &other) const + { + return dop_ == other.dop_ + && plan_id_ == other.plan_id_ + && exec_id_ == other.exec_id_ + && session_id_ == other.session_id_ + && database_id_ == other.database_id_ + && memcmp(sql_id_, other.sql_id_, common::OB_MAX_SQL_ID_LENGTH + 1) + && op_id_ == other.op_id_ + && input_rows_ == other.input_rows_ + && input_width_ == other.input_width_ + && disable_auto_mem_mgr_ == other.disable_auto_mem_mgr_; + } + + void set(int64_t dop, + int64_t plan_id, + int64_t exec_id, + int64_t session_id, + uint64_t database_id, + const char *sql_id, + uint64_t op_id, + int64_t input_rows, + int64_t input_width, + bool disable_auto_mem_mgr) + { + dop_ = dop; + plan_id_ = plan_id; + exec_id_ = exec_id; + session_id_ = session_id; + database_id_ = database_id; + op_id_ = op_id; + input_rows_ = input_rows; + input_width_ = input_width; + disable_auto_mem_mgr_ = disable_auto_mem_mgr; + if (OB_ISNULL(sql_id)) { + sql_id_[0] = '\0'; + } else { + memcpy(sql_id_, sql_id, OB_MAX_SQL_ID_LENGTH); + sql_id_[OB_MAX_SQL_ID_LENGTH] = '\0'; + } + } + + int64_t get_dop() { return dop_; } + int64_t get_plan_id() { return plan_id_; } + int64_t get_exec_id() { return exec_id_; } + int64_t get_session_id() { return session_id_; } + uint64_t get_database_id() { return database_id_; } + const char *get_sql_id() { return sql_id_; }; + uint64_t get_op_id() { return op_id_; } + int64_t get_input_rows() { return input_rows_; } + int64_t get_input_width() { return input_width_; } + bool get_disable_auto_mem_mgr() { return disable_auto_mem_mgr_; } + + TO_STRING_KV(K_(dop), K_(plan_id), K_(exec_id), K_(session_id), K_(sql_id), K_(database_id)); +public: + int64_t dop_; + int64_t plan_id_; + int64_t exec_id_; + int64_t session_id_; + uint64_t database_id_; + char sql_id_[common::OB_MAX_SQL_ID_LENGTH + 1]; + uint64_t op_id_; + int64_t input_rows_; + int64_t input_width_; + bool disable_auto_mem_mgr_; +}; + class ObDtlSqcInfo { public: @@ -138,7 +228,8 @@ public: sqc_id_(common::OB_INVALID_ID), enable_channel_sync_(false), register_dm_info_(), - row_meta_() + row_meta_(), + op_info_() {} ObDtlLinkedBuffer(char * buf, int64_t size) : buf_(buf), size_(size), pos_(), is_data_msg_(false), seq_no_(0), tenant_id_(0), @@ -149,11 +240,12 @@ public: sqc_id_(common::OB_INVALID_ID), enable_channel_sync_(false), register_dm_info_(), - row_meta_() + row_meta_(), + op_info_() {} - ~ObDtlLinkedBuffer() { reset_batch_info(); } TO_STRING_KV(K_(size), K_(pos), K_(is_data_msg), K_(seq_no), K_(tenant_id), K_(allocated_chid), - K_(is_eof), K_(timeout_ts), K(msg_type_), K_(flags), K(is_bcast()), K_(rows_cnt), K_(enable_channel_sync)); + K_(is_eof), K_(timeout_ts), K(msg_type_), K_(flags), K(is_bcast()), K_(rows_cnt), K_(enable_channel_sync), + K_(dfo_key), K_(op_info)); ObDtlLinkedBuffer *next() const { return reinterpret_cast(next_); @@ -295,6 +387,7 @@ public: dst->enable_channel_sync_ = src.enable_channel_sync_; dst->register_dm_info_ = src.register_dm_info_; dst->row_meta_ = src.row_meta_; + dst->op_info_ = src.op_info_; } void shallow_copy(const ObDtlLinkedBuffer &src) @@ -315,6 +408,7 @@ public: enable_channel_sync_ = src.enable_channel_sync_; register_dm_info_ = src.register_dm_info_; row_meta_ = src.row_meta_; + op_info_ = src.op_info_; } OB_INLINE ObDtlDfoKey &get_dfo_key() { @@ -329,6 +423,15 @@ public: { return dfo_key_.is_valid(); } + + OB_INLINE ObDtlOpInfo &get_op_info() { + return op_info_; + } + + OB_INLINE void set_op_info(ObDtlOpInfo &op_info) { + op_info_ = op_info; + } + OB_INLINE int64_t allocated_chid() const { return allocated_chid_; } @@ -368,7 +471,47 @@ public: int64_t get_dfo_id() { return dfo_id_; } int64_t get_sqc_id() { return sqc_id_; } RowMeta &get_row_meta() { return row_meta_; } + uint64_t get_px_sequence_id() { return dfo_key_.px_sequence_id_; } + int64_t get_dop() { return op_info_.dop_; } + void set_dop(int64_t dop) { op_info_.dop_ = dop; } + + int64_t get_plan_id() { return op_info_.plan_id_; } + void set_plan_id(int64_t plan_id) { op_info_.plan_id_ = plan_id; } + + int64_t get_exec_id() { return op_info_.exec_id_; } + void set_exec_id(int64_t exec_id) { op_info_.exec_id_ = exec_id; } + + int64_t get_session_id() { return op_info_.session_id_; } + void set_session_id(int64_t session_id) { op_info_.session_id_ = session_id; } + + uint64_t get_database_id() { return op_info_.database_id_; } + void set_database_id(uint64_t database_id) { op_info_.database_id_ = database_id; } + + const char *get_sql_id() { return op_info_.sql_id_; }; + void set_sql_id(const char *sql_id) { + if (OB_ISNULL(sql_id)) { + op_info_.sql_id_[0] = '\0'; + } else { + memcpy(op_info_.sql_id_, sql_id, OB_MAX_SQL_ID_LENGTH); + op_info_.sql_id_[OB_MAX_SQL_ID_LENGTH] = '\0'; + } + } + + uint64_t get_op_id() { return op_info_.op_id_; } + void set_op_id(uint64_t op_id) { op_info_.op_id_ = op_id; } + + int64_t get_input_rows() { return op_info_.input_rows_; } + void set_input_rows(int64_t input_rows) { op_info_.input_rows_ = input_rows; } + + int64_t get_input_width() { return op_info_.input_width_; } + void set_input_width(int64_t input_width) { op_info_.input_width_ = input_width; } + + bool get_disable_auto_mem_mgr() { return op_info_.disable_auto_mem_mgr_; } + void set_disable_auto_mem_mgr(bool disable_auto_mem_mgr) + { + op_info_.disable_auto_mem_mgr_ = disable_auto_mem_mgr; + } private: /* @@ -437,6 +580,7 @@ The memory layout is as below: bool enable_channel_sync_; common::ObRegisterDmInfo register_dm_info_; RowMeta row_meta_; + ObDtlOpInfo op_info_; }; } // dtl diff --git a/src/sql/engine/basic/ob_chunk_datum_store.cpp b/src/sql/engine/basic/ob_chunk_datum_store.cpp index 306f551af2..e56d431165 100644 --- a/src/sql/engine/basic/ob_chunk_datum_store.cpp +++ b/src/sql/engine/basic/ob_chunk_datum_store.cpp @@ -609,9 +609,7 @@ void ObChunkDatumStore::reset() } io_.fd_ = -1; } - file_size_ = 0; n_block_in_file_ = 0; - while (!blocks_.is_empty()) { Block *item = blocks_.remove_first(); mem_hold_ -= item->get_buffer()->mem_size(); @@ -636,6 +634,13 @@ void ObChunkDatumStore::reset() allocator_->free(item); } + if(OB_SUCC(ret)) { + if (nullptr != callback_) { + callback_->dumped(-file_size_); + } + } + file_size_ = 0; + if (NULL != batch_ctx_) { allocator_->free(batch_ctx_); batch_ctx_ = NULL; diff --git a/src/sql/engine/basic/ob_chunk_row_store.cpp b/src/sql/engine/basic/ob_chunk_row_store.cpp index c6271fa4c8..1f291c014a 100644 --- a/src/sql/engine/basic/ob_chunk_row_store.cpp +++ b/src/sql/engine/basic/ob_chunk_row_store.cpp @@ -293,7 +293,6 @@ void ObChunkRowStore::reset() } io_.fd_ = -1; } - file_size_ = 0; n_block_in_file_ = 0; while (!blocks_.is_empty()) { @@ -322,6 +321,13 @@ void ObChunkRowStore::reset() n_blocks_ = 0; row_cnt_ = 0; + if(OB_SUCC(ret)) { + if (nullptr != callback_) { + callback_->dumped(-file_size_); + } + } + file_size_ = 0; + if (NULL != projector_) { allocator_->free(projector_); } diff --git a/src/sql/engine/basic/ob_temp_table_insert_op.cpp b/src/sql/engine/basic/ob_temp_table_insert_op.cpp index 473b21ba23..3606b02b0f 100644 --- a/src/sql/engine/basic/ob_temp_table_insert_op.cpp +++ b/src/sql/engine/basic/ob_temp_table_insert_op.cpp @@ -331,7 +331,7 @@ int ObTempTableInsertOp::insert_chunk_row_store() } else { dtl_int_key.channel_id_ = interm_result_id; dtl_int_key.start_time_ = oceanbase::common::ObTimeUtility::current_time(); - dtl_int_key.time_us_ = phy_plan_ctx->get_timeout_timestamp(); + dtl_int_key.timeout_ts_ = phy_plan_ctx->get_timeout_timestamp(); row_store->set_eof(true); //chunk row store不需要管理dump逻辑 row_store->is_read_ = true; @@ -377,7 +377,7 @@ int ObTempTableInsertOp::clear_all_datum_store() for (int64_t i = 0; OB_SUCC(ret) && i < all_datum_store_.count(); ++i) { ObDTLIntermResultInfo *datum_store = all_datum_store_.at(i); if (NULL != datum_store) { - dtl::ObDTLIntermResultManager::dec_interm_result_ref_count(datum_store); + MTL(dtl::ObDTLIntermResultManager*)->dec_interm_result_ref_count(datum_store); } } all_datum_store_.reset(); diff --git a/src/sql/engine/basic/ob_temp_table_insert_vec_op.cpp b/src/sql/engine/basic/ob_temp_table_insert_vec_op.cpp index 6ba2c3fd7b..3e34fad761 100644 --- a/src/sql/engine/basic/ob_temp_table_insert_vec_op.cpp +++ b/src/sql/engine/basic/ob_temp_table_insert_vec_op.cpp @@ -270,7 +270,7 @@ int ObTempTableInsertVecOp::insert_interm_result_info() } else { dtl_int_key.channel_id_ = interm_result_id; dtl_int_key.start_time_ = oceanbase::common::ObTimeUtility::current_time(); - dtl_int_key.time_us_ = phy_plan_ctx->get_timeout_timestamp(); + dtl_int_key.timeout_ts_ = phy_plan_ctx->get_timeout_timestamp(); cur_interm_res_info->set_eof(true); // The chunk row store does not require managing the dump logic. cur_interm_res_info->is_read_ = true; @@ -317,7 +317,7 @@ int ObTempTableInsertVecOp::clear_all_interm_res_info() for (int64_t i = 0; OB_SUCC(ret) && i < all_interm_res_info_.count(); ++i) { ObDTLIntermResultInfo *col_store = all_interm_res_info_.at(i); if (NULL != col_store) { - dtl::ObDTLIntermResultManager::dec_interm_result_ref_count(col_store); + MTL(dtl::ObDTLIntermResultManager*)->dec_interm_result_ref_count(col_store); } } all_interm_res_info_.reset(); diff --git a/src/sql/engine/expr/ob_expr_autoinc_nextval.cpp b/src/sql/engine/expr/ob_expr_autoinc_nextval.cpp index 09a469cfd4..b5e8333522 100644 --- a/src/sql/engine/expr/ob_expr_autoinc_nextval.cpp +++ b/src/sql/engine/expr/ob_expr_autoinc_nextval.cpp @@ -357,7 +357,10 @@ int ObExprAutoincNextval::generate_autoinc_value(const ObSQLSessionInfo &my_sess } } } - + if (OB_UNLIKELY(OB_DATA_OUT_OF_RANGE == ret) && !is_strict_mode(my_session.get_sql_mode())) { + ret = OB_SUCCESS; + value = ObAutoincrementService::get_max_value(autoinc_param->autoinc_col_type_); + } if (OB_SUCC(ret)) { new_val = value; plan_ctx->set_autoinc_id_tmp(value); diff --git a/src/sql/engine/expr/ob_expr_pl_seq_nextval.cpp b/src/sql/engine/expr/ob_expr_pl_seq_nextval.cpp index 43abc3adae..331bb8a92f 100644 --- a/src/sql/engine/expr/ob_expr_pl_seq_nextval.cpp +++ b/src/sql/engine/expr/ob_expr_pl_seq_nextval.cpp @@ -108,11 +108,21 @@ int ObExprPLSeqNextval::eval_pl_seq_next_val( LOG_WARN("fail get sequence schema", K(seq_id), K(ret)); } else { ObSequenceValue seq_value; - if (OB_FAIL(seq_cache.nextval(*seq_schema, - alloc_guard.get_allocator(), - seq_value))) { - LOG_WARN("fail get nextval for seq", K(tenant_id), K(seq_id), K(ret)); - } else if (OB_FAIL(session->set_sequence_value(tenant_id, seq_id, seq_value))) { + ObAutoincrementService &auto_service = ObAutoincrementService::get_instance(); + if (seq_schema->get_order_flag() + && seq_schema->get_cache_order_mode() == NEW_ACTION) { + if (OB_FAIL(auto_service.get_handle(*seq_schema, seq_value))) { + LOG_WARN("fail get nextval from rpc for seq", K(tenant_id), K(seq_id), K(ret)); + } + } else { + if (OB_FAIL(seq_cache.nextval(*seq_schema, alloc_guard.get_allocator(), seq_value))) { + LOG_WARN("fail get nextval for seq", K(tenant_id), K(seq_id), K(ret)); + } + } + + if (OB_FAIL(ret)) { + // do nothing + }else if (OB_FAIL(session->set_sequence_value(tenant_id, seq_id, seq_value))) { LOG_WARN("save seq_value to session as currval for later read fail", K(tenant_id), K(seq_id), K(seq_value), K(ret)); } else { /*do nothing*/ } diff --git a/src/sql/engine/ob_physical_plan_ctx.cpp b/src/sql/engine/ob_physical_plan_ctx.cpp index 3b7118966d..fc8f1c9f34 100644 --- a/src/sql/engine/ob_physical_plan_ctx.cpp +++ b/src/sql/engine/ob_physical_plan_ctx.cpp @@ -123,7 +123,9 @@ ObPhysicalPlanCtx::ObPhysicalPlanCtx(common::ObIAllocator &allocator) tx_id_(0), tm_sessid_(0), hint_xa_trans_stop_check_lock_(false), - main_xa_trans_branch_(false) + main_xa_trans_branch_(false), + total_memstore_read_row_count_(0), + total_ssstore_read_row_count_(0) { } diff --git a/src/sql/engine/ob_physical_plan_ctx.h b/src/sql/engine/ob_physical_plan_ctx.h index 4eb630a458..faf1406876 100644 --- a/src/sql/engine/ob_physical_plan_ctx.h +++ b/src/sql/engine/ob_physical_plan_ctx.h @@ -99,6 +99,8 @@ public: virtual ~ObPhysicalPlanCtx(); void destroy() { + total_memstore_read_row_count_ = 0; + total_ssstore_read_row_count_ = 0; // Member variables that need to request additional memory // with another allocator should call destroy here. subschema_ctx_.destroy(); @@ -242,6 +244,22 @@ public: { affected_rows_ += affected_rows; } + inline void add_total_memstore_read_row_count(int64_t v) + { + total_memstore_read_row_count_ += v; + } + inline void add_total_ssstore_read_row_count(int64_t v) + { + total_ssstore_read_row_count_ += v; + } + inline int64_t get_total_memstore_read_row_count() + { + return total_memstore_read_row_count_; + } + inline int64_t get_total_ssstore_read_row_count() + { + return total_ssstore_read_row_count_; + } int64_t get_found_rows() const { return found_rows_; @@ -647,6 +665,8 @@ private: bool hint_xa_trans_stop_check_lock_; // for dblink to stop check stmt lock in xa trans bool main_xa_trans_branch_; // for dblink to indicate weather this sql is executed in main_xa_trans_branch ObSEArray dblink_ids_; + int64_t total_memstore_read_row_count_; + int64_t total_ssstore_read_row_count_; }; } diff --git a/src/sql/engine/ob_sql_mem_mgr_processor.cpp b/src/sql/engine/ob_sql_mem_mgr_processor.cpp index 7ea3df3657..3ba1737e78 100644 --- a/src/sql/engine/ob_sql_mem_mgr_processor.cpp +++ b/src/sql/engine/ob_sql_mem_mgr_processor.cpp @@ -27,7 +27,7 @@ int ObSqlMemMgrProcessor::init( int64_t cache_size, const ObPhyOperatorType op_type, const uint64_t op_id, - ObExecContext *exec_ctx) + ObSqlProfileExecInfo exec_info) { int ret = OB_SUCCESS; bool tmp_enable_auto_mem_mgr = false; @@ -37,26 +37,18 @@ int ObSqlMemMgrProcessor::init( tenant_id_ = tenant_id; profile_.set_operator_type(op_type); profile_.set_operator_id(op_id); - profile_.set_exec_ctx(exec_ctx); + profile_.set_exec_info(exec_info); const int64_t DEFAULT_CACHE_SIZE = 2 * 1024 * 1024; if (cache_size < 0) { LOG_WARN("unexpected cache size got", K(lbt()), K(cache_size), K(op_id), K(op_type)); cache_size = DEFAULT_CACHE_SIZE; } - if (OB_ISNULL(exec_ctx)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("failed to get exec ctx", K(ret)); - } else if (OB_ISNULL(allocator)) { + if (OB_ISNULL(allocator)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("failed to get allocator", K(ret)); - } else if (OB_FAIL(profile_.set_exec_info(*exec_ctx))) { - LOG_WARN("failed to set exec info", K(ret)); } else if (OB_FAIL(alloc_dir_id(dir_id_))) { } else if (OB_NOT_NULL(sql_mem_mgr)) { - if (OB_NOT_NULL(exec_ctx->get_physical_plan_ctx()) - && OB_NOT_NULL(exec_ctx->get_physical_plan_ctx()->get_phy_plan())) { - profile_.disable_auto_mem_mgr_ = exec_ctx->get_physical_plan_ctx()->get_phy_plan()->is_disable_auto_memory_mgr(); - } + profile_.disable_auto_mem_mgr_ = exec_info.get_disable_auto_mem_mgr(); if (sql_mem_mgr->enable_auto_memory_mgr()) { tmp_enable_auto_mem_mgr = true; if (profile_.get_auto_policy()) { @@ -74,7 +66,7 @@ int ObSqlMemMgrProcessor::init( } else { // first time to register profile_.init(cache_size, OB_MALLOC_MIDDLE_BLOCK_SIZE); - LOG_DEBUG("trace register work area profile", K(profile_.get_cache_size())); + LOG_TRACE("trace register work area profile", K(profile_.get_cache_size())); } // 每次初始化都认为是未注册 profile_.set_expect_size(OB_INVALID_ID); @@ -104,7 +96,7 @@ int ObSqlMemMgrProcessor::init( int64_t max_mem_size = MAX_SQL_MEM_SIZE; if (OB_FAIL(ret)) { } else if (OB_FAIL(ObSqlWorkareaUtil::get_workarea_size( - profile_.get_work_area_type(), tenant_id_, exec_ctx, max_mem_size))) { + profile_.get_work_area_type(), tenant_id_, exec_info, max_mem_size))) { LOG_WARN("failed to get workarea size", K(ret), K(tenant_id_), K(max_mem_size)); } if (!profile_.get_auto_policy()) { @@ -191,6 +183,9 @@ int ObSqlMemMgrProcessor::update_used_mem_size(int64_t used_size) { int ret = OB_SUCCESS; int64_t delta_size = used_size - profile_.mem_used_; + if (OB_NOT_NULL(op_monitor_info_)) { + op_monitor_info_->update_memory(delta_size); + } if (delta_size > 0) { if (OB_NOT_NULL(sql_mem_mgr_) && OB_NOT_NULL(mem_callback_)) { mem_callback_->alloc(delta_size); @@ -224,7 +219,7 @@ int ObSqlMemMgrProcessor::try_upgrade_auto_mgr(ObIAllocator *allocator, int64_t } else if (OB_FAIL(init(allocator, tenant_id_, max_area_size * (EXTEND_RATIO + 100) /100, profile_.get_operator_type(), - profile_.get_operator_id(), profile_.get_exec_ctx()))) { + profile_.get_operator_id(), profile_.get_exec_info()))) { LOG_WARN("failed to upgrade sql memory manager", K(ret)); } else if (is_auto_mgr()) { // 由于之前未注册,所以对内存统计不会反应到sql mem manager中,但现在注册了,则需要重新更新下内存值 @@ -335,20 +330,42 @@ int ObSqlWorkareaUtil::get_workarea_size(const ObSqlWorkAreaType wa_type, int64_t &value) { int ret = OB_SUCCESS; + if (OB_ISNULL(exec_ctx)) { + if (OB_FAIL(get_workarea_size(wa_type, tenant_id, value, nullptr))) { + LOG_WARN("Fail to get workarea size", K(ret)); + } + } else if (OB_FAIL(get_workarea_size(wa_type, tenant_id, value, exec_ctx->get_my_session()))) { + LOG_WARN("Fail to get workarea size", K(ret)); + } + return ret; +} - if (NULL != exec_ctx) { - if (OB_ISNULL(exec_ctx->get_my_session())) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpected exec_ctx or session", K(ret), K(wa_type), K(tenant_id), KP(exec_ctx)); +int ObSqlWorkareaUtil::get_workarea_size(const ObSqlWorkAreaType wa_type, + const int64_t tenant_id, + ObSqlProfileExecInfo &exec_info, + int64_t &value) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(get_workarea_size(wa_type, tenant_id, value, exec_info.get_my_session()))) { + LOG_WARN("Fail to get workarea size", K(ret)); + } + return ret; +} + +int ObSqlWorkareaUtil::get_workarea_size(const ObSqlWorkAreaType wa_type, + const int64_t tenant_id, + int64_t &value, + ObSQLSessionInfo *session) +{ + int ret = OB_SUCCESS; + if (OB_NOT_NULL(session)) { + if (HASH_WORK_AREA == wa_type) { + value = session->get_tenant_hash_area_size(); + } else if (SORT_WORK_AREA == wa_type) { + value = session->get_tenant_sort_area_size(); } else { - if (HASH_WORK_AREA == wa_type) { - value = exec_ctx->get_my_session()->get_tenant_hash_area_size(); - } else if (SORT_WORK_AREA == wa_type) { - value = exec_ctx->get_my_session()->get_tenant_sort_area_size(); - } else { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpected status: workarea type", K(wa_type), K(tenant_id)); - } + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected status: workarea type", K(wa_type), K(tenant_id)); } } else { ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id)); @@ -361,15 +378,12 @@ int ObSqlWorkareaUtil::get_workarea_size(const ObSqlWorkAreaType wa_type, ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected status: workarea type", K(wa_type), K(tenant_id)); } - LOG_DEBUG("debug workarea size", K(value), K(tenant_id), K(lbt())); } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("failed to init tenant config", K(tenant_id), K(ret)); } } - LOG_DEBUG("debug workarea size", K(value), K(tenant_id), K(lbt())); - return ret; } diff --git a/src/sql/engine/ob_sql_mem_mgr_processor.h b/src/sql/engine/ob_sql_mem_mgr_processor.h index 1abd488d16..3b3916b03c 100644 --- a/src/sql/engine/ob_sql_mem_mgr_processor.h +++ b/src/sql/engine/ob_sql_mem_mgr_processor.h @@ -26,15 +26,22 @@ private: using PredFunc = std::function; public: ObSqlMemMgrProcessor(ObSqlWorkAreaProfile &profile, ObMonitorNode &op_monitor_info) : - profile_(profile), op_monitor_info_(op_monitor_info), + profile_(profile), op_monitor_info_(&op_monitor_info), sql_mem_mgr_(nullptr), mem_callback_(nullptr), tenant_id_(OB_INVALID_ID), - periodic_cnt_(1024), - origin_max_mem_size_(0), default_available_mem_size_(0), is_auto_mgr_(false), dir_id_(0), + periodic_cnt_(1024), origin_max_mem_size_(0), default_available_mem_size_(0), + is_auto_mgr_(false), dir_id_(0), dummy_ptr_(nullptr), dummy_alloc_(nullptr) { // trace memory dump - op_monitor_info_.otherstat_6_id_ = ObSqlMonitorStatIds::MEMORY_DUMP; + op_monitor_info_->otherstat_6_id_ = ObSqlMonitorStatIds::MEMORY_DUMP; } + + ObSqlMemMgrProcessor(ObSqlWorkAreaProfile &profile) : + profile_(profile), op_monitor_info_(nullptr), + sql_mem_mgr_(nullptr), mem_callback_(nullptr), tenant_id_(OB_INVALID_ID), + periodic_cnt_(1024), origin_max_mem_size_(0), default_available_mem_size_(0), + is_auto_mgr_(false), dir_id_(0), + dummy_ptr_(nullptr), dummy_alloc_(nullptr) {} virtual ~ObSqlMemMgrProcessor() {} void set_sql_mem_mgr(ObTenantSqlMemoryManager *sql_mem_mgr) @@ -60,7 +67,8 @@ public: int64_t cache_size, const ObPhyOperatorType op_type, const uint64_t op_id, - ObExecContext *exec_ctx); + ObSqlProfileExecInfo exec_info); + void destroy() { if (0 < profile_.mem_used_ && OB_NOT_NULL(mem_callback_)) { @@ -102,35 +110,43 @@ public: void alloc(int64_t size) { profile_.delta_size_ += size; - update_delta_size(profile_.delta_size_); + update_memory_delta_size(profile_.delta_size_); } void free(int64_t size) { profile_.delta_size_ -= size; - update_delta_size(profile_.delta_size_); + update_memory_delta_size(profile_.delta_size_); } - void dumped(int64_t size) + void dumped(int64_t delta_size) { - profile_.dumped_size_ += size; - op_monitor_info_.otherstat_6_value_ += size; + profile_.dumped_size_ += delta_size; + if (OB_NOT_NULL(op_monitor_info_)) { + op_monitor_info_->otherstat_6_value_ += delta_size; + op_monitor_info_->update_tempseg(delta_size); + } + profile_.max_dumped_size_ = MAX(profile_.max_dumped_size_, profile_.dumped_size_); if (OB_NOT_NULL(mem_callback_)) { - mem_callback_->dumped(size); + mem_callback_->dumped(delta_size); } } int64_t get_dumped_size() const { return profile_.dumped_size_; } + int64_t get_max_dumped_size() const { return profile_.max_dumped_size_; } void reset_delta_size() { profile_.delta_size_ = 0; } int64_t get_mem_used() const { return profile_.mem_used_; } int64_t get_delta_size() const { return profile_.delta_size_; } int64_t get_data_size() const { return profile_.data_size_ + profile_.delta_size_; } - void update_delta_size(int64_t delta_size) + void update_memory_delta_size(int64_t delta_size) { if (delta_size > 0 && delta_size >= UPDATED_DELTA_SIZE) { if (OB_NOT_NULL(mem_callback_)) { mem_callback_->alloc(delta_size); } profile_.mem_used_ += delta_size; + if (OB_NOT_NULL(op_monitor_info_)) { + op_monitor_info_->update_memory(delta_size); + } if (profile_.max_mem_used_ < profile_.mem_used_) { profile_.max_mem_used_ = profile_.mem_used_; } @@ -141,6 +157,9 @@ public: mem_callback_->free(-delta_size); } profile_.mem_used_ += delta_size; + if (OB_NOT_NULL(op_monitor_info_)) { + op_monitor_info_->update_memory(delta_size); + } profile_.delta_size_ = 0; profile_.data_size_ += delta_size; } @@ -166,7 +185,7 @@ private: static const int64_t UPDATED_DELTA_SIZE = 1 * 1024 * 1024; static const int64_t EXTEND_RATIO = 10; ObSqlWorkAreaProfile &profile_; - ObMonitorNode &op_monitor_info_; + ObMonitorNode *op_monitor_info_; ObTenantSqlMemoryManager *sql_mem_mgr_; ObSqlMemoryCallback *mem_callback_; uint64_t tenant_id_; @@ -188,6 +207,20 @@ public: ObExecContext *exec_ctx, int64_t &value ); + + static int get_workarea_size( + const ObSqlWorkAreaType wa_type, + const int64_t tenant_id, + ObSqlProfileExecInfo &exec_info, + int64_t &value + ); + + static int get_workarea_size( + const ObSqlWorkAreaType wa_type, + const int64_t tenant_id, + int64_t &value, + ObSQLSessionInfo *session = nullptr + ); }; } // sql diff --git a/src/sql/engine/ob_tenant_sql_memory_manager.cpp b/src/sql/engine/ob_tenant_sql_memory_manager.cpp index 4cc8b2e8c2..37bcd86dff 100644 --- a/src/sql/engine/ob_tenant_sql_memory_manager.cpp +++ b/src/sql/engine/ob_tenant_sql_memory_manager.cpp @@ -43,57 +43,34 @@ const int64_t ObSqlWorkAreaProfile::MIN_BOUND_SIZE[ObSqlWorkAreaType::MAX_TYPE] int64_t ObSqlWorkAreaProfile::get_dop() { - return dop_; + return exec_info_.get_dop(); } uint64_t ObSqlWorkAreaProfile::get_plan_id() { - return plan_id_; + return exec_info_.get_plan_id(); } uint64_t ObSqlWorkAreaProfile::get_exec_id() { - return exec_id_; + return exec_info_.get_exec_id(); } const char* ObSqlWorkAreaProfile::get_sql_id() { - return sql_id_; + return exec_info_.get_sql_id(); } uint64_t ObSqlWorkAreaProfile::get_session_id() { - return session_id_; + return exec_info_.get_session_id(); } uint64_t ObSqlWorkAreaProfile::get_db_id() { - return db_id_; + return exec_info_.get_db_id(); } -int ObSqlWorkAreaProfile::set_exec_info(ObExecContext &exec_ctx) - { - int ret = OB_SUCCESS; - dop_ = ObPxSqcUtil::get_actual_worker_count(&exec_ctx); - plan_id_ = ObPxSqcUtil::get_plan_id(&exec_ctx); - exec_id_ = ObPxSqcUtil::get_exec_id(&exec_ctx); - session_id_ = ObPxSqcUtil::get_session_id(&exec_ctx); - ObPhysicalPlanCtx *plan_ctx = exec_ctx.get_physical_plan_ctx(); - if (OB_NOT_NULL(plan_ctx) && OB_NOT_NULL(plan_ctx->get_phy_plan())) { - if (nullptr == plan_ctx->get_phy_plan()->get_sql_id()) { - sql_id_[0] = '\0'; - } else { - memcpy(sql_id_, plan_ctx->get_phy_plan()->get_sql_id(), OB_MAX_SQL_ID_LENGTH); - sql_id_[OB_MAX_SQL_ID_LENGTH] = '\0'; - } - } - ObSQLSessionInfo *sql_session = exec_ctx.get_my_session(); - if (OB_NOT_NULL(sql_session)) { - db_id_ = sql_session->get_database_id(); - } - return ret; - } - //////////////////////////////////////////////////////////////////////////////////// int ObSqlWorkAreaIntervalStat::analyze_profile( ObSqlWorkAreaProfile &profile, @@ -657,7 +634,7 @@ int ObTenantSqlMemoryManager::fill_workarea_stat( wa_stat.active_avg_time_ = (active_avg_time + (ObTimeUtility::current_time() - profile.get_active_time())) / wa_stat.get_total_executions(); - wa_stat.last_temp_size_ = profile.get_dumped_size(); + wa_stat.last_temp_size_ = profile.get_max_dumped_size(); if (wa_stat.max_temp_size_ < wa_stat.last_temp_size_) { wa_stat.max_temp_size_ = wa_stat.last_temp_size_; } @@ -774,7 +751,8 @@ int ObTenantSqlMemoryManager::new_and_fill_workarea_stat( int ObTenantSqlMemoryManager::collect_workarea_stat(ObSqlWorkAreaProfile &profile) { int ret = OB_SUCCESS; - if (profile.has_exec_ctx()) { + if (profile.has_exec_info() || + profile.get_operator_type() == oceanbase::sql::PHY_PX_FIFO_RECEIVE) { ObSqlWorkAreaStat::WorkareaKey workarea_key( profile.get_plan_id(), profile.get_operator_id(), @@ -1359,5 +1337,45 @@ int ObTenantSqlMemoryManager::get_workarea_memory_info( return ret; } +ObSqlProfileExecInfo::ObSqlProfileExecInfo(ObExecContext *exec_ctx) +{ + dop_ = ObPxSqcUtil::get_actual_worker_count(exec_ctx); + plan_id_ = ObPxSqcUtil::get_plan_id(exec_ctx); + exec_id_ = ObPxSqcUtil::get_exec_id(exec_ctx); + session_id_ = ObPxSqcUtil::get_session_id(exec_ctx); + ObPhysicalPlanCtx *plan_ctx = exec_ctx->get_physical_plan_ctx(); + if (OB_NOT_NULL(plan_ctx) && OB_NOT_NULL(plan_ctx->get_phy_plan())) { + disable_auto_mem_mgr_ = plan_ctx->get_phy_plan()->is_disable_auto_memory_mgr(); + if (nullptr == plan_ctx->get_phy_plan()->get_sql_id()) { + sql_id_[0] = '\0'; + } else { + memcpy(sql_id_, plan_ctx->get_phy_plan()->get_sql_id(), OB_MAX_SQL_ID_LENGTH); + sql_id_[OB_MAX_SQL_ID_LENGTH] = '\0'; + } + } + my_session_ = exec_ctx->get_my_session(); + if (OB_NOT_NULL(my_session_)) { + db_id_ = my_session_->get_database_id(); + } +} + +ObSqlProfileExecInfo::ObSqlProfileExecInfo(dtl::ObDtlLinkedBuffer *buffer) : + my_session_(nullptr) +{ + dop_ = buffer->get_dop(); + plan_id_ = buffer->get_plan_id(); + exec_id_ = buffer->get_exec_id(); + session_id_ = buffer->get_session_id(); + db_id_ = buffer->get_database_id(); + disable_auto_mem_mgr_ = buffer->get_disable_auto_mem_mgr(); + if (*(buffer->get_sql_id()) == '\0') { + sql_id_[0] = '\0'; + } else { + MEMCPY(sql_id_, buffer->get_sql_id(), OB_MAX_SQL_ID_LENGTH); + sql_id_[OB_MAX_SQL_ID_LENGTH] = '\0'; + } +} + + } // sql } // oceanbase diff --git a/src/sql/engine/ob_tenant_sql_memory_manager.h b/src/sql/engine/ob_tenant_sql_memory_manager.h index 47cdc2fc63..610d89cc25 100644 --- a/src/sql/engine/ob_tenant_sql_memory_manager.h +++ b/src/sql/engine/ob_tenant_sql_memory_manager.h @@ -20,6 +20,7 @@ #include "sql/engine/basic/ob_chunk_row_store.h" #include "sql/engine/ob_phy_operator_type.h" #include "sql/engine/ob_exec_context.h" +#include "sql/dtl/ob_dtl_linked_buffer.h" namespace oceanbase { namespace sql { @@ -31,21 +32,64 @@ enum ObSqlWorkAreaType MAX_TYPE }; +class ObSqlProfileExecInfo { +public: + ObSqlProfileExecInfo() : dop_(-1), plan_id_(-1), exec_id_(-1), + session_id_(-1), db_id_(-1), my_session_(nullptr), + disable_auto_mem_mgr_(false) { + sql_id_[0] = '\0'; + } + ObSqlProfileExecInfo(const ObSqlProfileExecInfo& other) + : dop_(other.dop_), plan_id_(other.plan_id_), exec_id_(other.exec_id_), + session_id_(other.session_id_), db_id_(other.db_id_), + my_session_(other.my_session_), + disable_auto_mem_mgr_(other.disable_auto_mem_mgr_) { + if (*(other.get_sql_id()) == '\0') { + sql_id_[0] = '\0'; + } else { + MEMCPY(sql_id_, other.get_sql_id(), OB_MAX_SQL_ID_LENGTH); + sql_id_[OB_MAX_SQL_ID_LENGTH] = '\0'; + } + } + ObSqlProfileExecInfo(ObExecContext *exec_ctx); + ObSqlProfileExecInfo(dtl::ObDtlLinkedBuffer *buffer); + + int64_t get_dop() { return dop_; } + int64_t get_plan_id() { return plan_id_; } + int64_t get_exec_id() { return exec_id_; } + int64_t get_session_id() { return session_id_; } + uint64_t get_db_id() { return db_id_; } + const char *get_sql_id() const { return sql_id_; }; + ObSQLSessionInfo *get_my_session() { return my_session_; } + bool get_disable_auto_mem_mgr() { return disable_auto_mem_mgr_; } + + TO_STRING_KV(K_(dop), K_(plan_id), K_(exec_id), K_(session_id), K_(db_id), K_(sql_id), K_(disable_auto_mem_mgr)); + +private: + int64_t dop_; + int64_t plan_id_; + int64_t exec_id_; + int64_t session_id_; + uint64_t db_id_; + char sql_id_[common::OB_MAX_SQL_ID_LENGTH + 1]; + ObSQLSessionInfo *my_session_; + bool disable_auto_mem_mgr_; +}; + class ObSqlWorkAreaProfile : public common::ObDLinkBase { public: ObSqlWorkAreaProfile(ObSqlWorkAreaType type) : ObDLinkBase(), - random_id_(0), type_(type), op_type_(PHY_INVALID), op_id_(UINT64_MAX), exec_ctx_(nullptr), + random_id_(0), type_(type), op_type_(PHY_INVALID), op_id_(UINT64_MAX), exec_info_(), min_size_(0), row_count_(0), input_size_(0), bucket_size_(0), chunk_size_(0), cache_size_(-1), one_pass_size_(0), expect_size_(OB_INVALID_ID), - global_bound_size_(INT64_MAX), dop_(-1), plan_id_(-1), exec_id_(-1), sql_id_(), db_id_(-1), - session_id_(-1), max_bound_(INT64_MAX), delta_size_(0), data_size_(0), + global_bound_size_(INT64_MAX), max_bound_(INT64_MAX), delta_size_(0), data_size_(0), max_mem_used_(0), mem_used_(0), - pre_mem_used_(0), dumped_size_(0), data_ratio_(0.5), active_time_(0), number_pass_(0), + pre_mem_used_(0), dumped_size_(0), max_dumped_size_(0), data_ratio_(0.5), + active_time_(0), number_pass_(0), calc_count_(0), disable_auto_mem_mgr_(false) { - sql_id_[0] = '\0'; ObRandom rand; random_id_ = rand.get(); } @@ -68,12 +112,12 @@ public: bucket_size_ = bucket_size; } - int set_exec_info(ObExecContext &exec_ctx); OB_INLINE void set_operator_type(ObPhyOperatorType op_type) { op_type_ = op_type; } OB_INLINE void set_operator_id(uint64_t op_id) { op_id_ = op_id; } - OB_INLINE void set_exec_ctx(ObExecContext *exec_ctx) { exec_ctx_ = exec_ctx; } - OB_INLINE ObExecContext* get_exec_ctx() const { return exec_ctx_; } - bool has_exec_ctx() { return nullptr != exec_ctx_; } + OB_INLINE void set_exec_ctx(ObExecContext *exec_ctx) { exec_info_ = exec_ctx; } + OB_INLINE void set_exec_info(const ObSqlProfileExecInfo &exec_info) { exec_info_ = exec_info; } + OB_INLINE const ObSqlProfileExecInfo& get_exec_info() const { return exec_info_; } + bool has_exec_info() { return exec_info_.get_plan_id() >= 0; } // one_pass_size = sqrt(cache_size * chunk_size) OB_INLINE void init(int64_t cache_size, int64_t chunk_size) @@ -134,6 +178,7 @@ public: int64_t get_max_mem_used() const { return max_mem_used_; } int64_t get_mem_used() const { return mem_used_; } int64_t get_dumped_size() const { return dumped_size_; } + int64_t get_max_dumped_size() const { return max_dumped_size_; } int64_t get_data_ratio() const { return data_ratio_; } OB_INLINE bool is_registered() const { return OB_NOT_NULL(get_next()) || OB_NOT_NULL(get_prev()); } @@ -167,7 +212,7 @@ private: ObSqlWorkAreaType type_; ObPhyOperatorType op_type_; uint64_t op_id_; - ObExecContext *exec_ctx_; + ObSqlProfileExecInfo exec_info_; int64_t min_size_; int64_t row_count_; int64_t input_size_; @@ -177,12 +222,6 @@ private: int64_t one_pass_size_; int64_t expect_size_; int64_t global_bound_size_; - int64_t dop_; - int64_t plan_id_; - int64_t exec_id_; - char sql_id_[common::OB_MAX_SQL_ID_LENGTH + 1]; - uint64_t db_id_; - int64_t session_id_; // 取 min(cache_size, global_bound_size) // sort场景,在global_bound_size比较大情况下,sort理论上有data和extra内存,data应该是one-pass size // 也就是expect_size @@ -195,6 +234,7 @@ public: int64_t mem_used_; int64_t pre_mem_used_; int64_t dumped_size_; + int64_t max_dumped_size_; double data_ratio_; public: // some statistics diff --git a/src/sql/engine/px/ob_dfo.h b/src/sql/engine/px/ob_dfo.h index bd3780d98b..31a91b9679 100644 --- a/src/sql/engine/px/ob_dfo.h +++ b/src/sql/engine/px/ob_dfo.h @@ -931,7 +931,9 @@ public: tx_desc_(NULL), is_use_local_thread_(false), fb_info_(), - err_msg_() + err_msg_(), + memstore_read_row_count_(0), + ssstore_read_row_count_(0) { } @@ -961,6 +963,8 @@ public: tx_desc_ = other.tx_desc_; is_use_local_thread_ = other.is_use_local_thread_; fb_info_.assign(other.fb_info_); + memstore_read_row_count_ = other.memstore_read_row_count_; + ssstore_read_row_count_ = other.ssstore_read_row_count_; return *this; } public: @@ -986,7 +990,9 @@ public: K_(interm_result_ids), K_(tx_desc), K_(is_use_local_thread), - K_(fb_info)); + K_(fb_info), + K_(memstore_read_row_count), + K_(ssstore_read_row_count)); dtl::ObDtlChannelInfo &get_sqc_channel_info() { return sqc_ch_info_; } dtl::ObDtlChannelInfo &get_task_channel_info() { return task_ch_info_; } void set_task_channel(dtl::ObDtlChannel *ch) { task_channel_ = ch; } @@ -1034,6 +1040,10 @@ public: ObExecFeedbackInfo &get_feedback_info() { return fb_info_; }; const ObPxUserErrorMsg &get_err_msg() const { return err_msg_; } ObPxUserErrorMsg &get_err_msg() { return err_msg_; } + void set_memstore_read_row_count(int64_t v) { memstore_read_row_count_ = v; } + void set_ssstore_read_row_count(int64_t v) { ssstore_read_row_count_ = v; } + int64_t get_memstore_read_row_count() const { return memstore_read_row_count_; } + int64_t get_ssstore_read_row_count() const { return ssstore_read_row_count_; } public: // 小于等于0表示设置了rc 值, task default ret值为1 static const int64_t TASK_DEFAULT_RET_VALUE = 1; @@ -1065,6 +1075,8 @@ public: bool is_use_local_thread_; ObExecFeedbackInfo fb_info_; //for feedback info ObPxUserErrorMsg err_msg_; // for error msg & warning msg + int64_t memstore_read_row_count_; // the count of row from mem + int64_t ssstore_read_row_count_; // the count of row from disk }; class ObPxRpcInitTaskArgs diff --git a/src/sql/engine/px/ob_px_rpc_processor.cpp b/src/sql/engine/px/ob_px_rpc_processor.cpp index 1bb573d7f1..ff5929188e 100644 --- a/src/sql/engine/px/ob_px_rpc_processor.cpp +++ b/src/sql/engine/px/ob_px_rpc_processor.cpp @@ -488,6 +488,13 @@ void ObFastInitSqcCB::on_timeout() interrupt_qc(ret); } +void ObFastInitSqcCB::log_warn_sqc_fail(int ret) +{ + // Do not change the follow log about px_obdiag_sqc_addr, becacue it will use in obdiag tool + LOG_WARN("init fast sqc cb async interrupt qc", K_(trace_id), K(timeout_ts_), K(interrupt_id_), + K(ret), "px_obdiag_sqc_addr", addr_); +} + int ObFastInitSqcCB::process() { // @@ -496,8 +503,7 @@ int ObFastInitSqcCB::process() int64_t cur_timestamp = ::oceanbase::common::ObTimeUtility::current_time(); if (timeout_ts_ - cur_timestamp > 0) { interrupt_qc(ret); - LOG_WARN("init fast sqc cb async interrupt qc", K_(trace_id), - K(addr_), K(timeout_ts_), K(interrupt_id_), K(ret)); + log_warn_sqc_fail(ret); } else { LOG_WARN("init fast sqc cb async timeout", K_(trace_id), K(addr_), K(timeout_ts_), K(cur_timestamp), K(ret)); diff --git a/src/sql/engine/px/ob_px_rpc_processor.h b/src/sql/engine/px/ob_px_rpc_processor.h index 3c2eb9dae9..7e48465b1d 100644 --- a/src/sql/engine/px/ob_px_rpc_processor.h +++ b/src/sql/engine/px/ob_px_rpc_processor.h @@ -169,6 +169,7 @@ public: virtual void set_args(const Request &arg) { UNUSED(arg); } int deal_with_rpc_timeout_err_safely(); void interrupt_qc(int err); + void log_warn_sqc_fail(int ret); private: common::ObAddr addr_; ObQueryRetryInfo *retry_info_; diff --git a/src/sql/engine/px/ob_px_scheduler.cpp b/src/sql/engine/px/ob_px_scheduler.cpp index 115310a6cf..9365faaae5 100644 --- a/src/sql/engine/px/ob_px_scheduler.cpp +++ b/src/sql/engine/px/ob_px_scheduler.cpp @@ -299,6 +299,12 @@ int ObPxMsgProc::on_sqc_finish_msg(ObExecContext &ctx, return ret; } +void ObPxMsgProc::log_warn_sqc_fail(int ret, const ObPxFinishSqcResultMsg &pkt, ObPxSqcMeta *sqc) +{ + // Do not change the follow log about px_obdiag_sqc_addr, becacue it will use in obdiag tool + LOG_WARN("sqc fail, abort qc", K(pkt), K(ret), "px_obdiag_sqc_addr", sqc->get_exec_addr()); +} + int ObPxMsgProc::process_sqc_finish_msg_once(ObExecContext &ctx, const ObPxFinishSqcResultMsg &pkt, ObPxSqcMeta *sqc, ObDfo *edge) { @@ -408,7 +414,7 @@ int ObPxMsgProc::process_sqc_finish_msg_once(ObExecContext &ctx, const ObPxFinis if (OB_SUCC(ret)) { if (OB_FAIL(pkt.rc_)) { DAS_CTX(ctx).get_location_router().save_cur_exec_status(pkt.rc_); - LOG_WARN("sqc fail, abort qc", K(pkt), K(ret), "sqc_addr", sqc->get_exec_addr()); + log_warn_sqc_fail(ret, pkt, sqc); } else { // pkt rc_ == OB_SUCCESS // 处理 dml + px 框架下的affected row @@ -419,6 +425,10 @@ int ObPxMsgProc::process_sqc_finish_msg_once(ObExecContext &ctx, const ObPxFinis } else { ctx.get_physical_plan_ctx()->add_affected_rows(pkt.sqc_affected_rows_); ctx.get_physical_plan_ctx()->add_px_dml_row_info(pkt.dml_row_info_); + ctx.get_physical_plan_ctx()->add_total_memstore_read_row_count( + pkt.sqc_memstore_row_read_count_); + ctx.get_physical_plan_ctx()->add_total_ssstore_read_row_count( + pkt.sqc_ssstore_row_read_count_); } } } diff --git a/src/sql/engine/px/ob_px_scheduler.h b/src/sql/engine/px/ob_px_scheduler.h index 63dd435520..be5cb9c674 100644 --- a/src/sql/engine/px/ob_px_scheduler.h +++ b/src/sql/engine/px/ob_px_scheduler.h @@ -243,6 +243,7 @@ public: int on_piece_msg(ObExecContext &ctx, const ObOptStatsGatherPieceMsg &pkt); void clean_dtl_interm_result(ObExecContext &ctx); // end DATAHUB msg processing + void log_warn_sqc_fail(int ret, const ObPxFinishSqcResultMsg &pkt, ObPxSqcMeta *sqc); private: int do_cleanup_dfo(ObDfo &dfo); int fast_dispatch_sqc(ObExecContext &exec_ctx, diff --git a/src/sql/engine/px/ob_px_sqc_proxy.cpp b/src/sql/engine/px/ob_px_sqc_proxy.cpp index 60d00fdc90..4f2147ffdb 100644 --- a/src/sql/engine/px/ob_px_sqc_proxy.cpp +++ b/src/sql/engine/px/ob_px_sqc_proxy.cpp @@ -444,6 +444,8 @@ int ObPxSQCProxy::report(int end_ret) const ObPxSqcMeta &sqc = sqc_arg.sqc_; ObPxFinishSqcResultMsg finish_msg; int64_t affected_rows = 0; + int64_t sqc_memstore_row_read_count = 0; + int64_t sqc_ssstore_row_read_count = 0; // 任意一个 task 失败,则意味着全部 task 失败 // 第一版暂不支持重试 int sqc_ret = OB_SUCCESS; @@ -463,6 +465,8 @@ int ObPxSQCProxy::report(int end_ret) const finish_msg.das_retry_rc_ = task.get_das_retry_rc(); } affected_rows += task.get_affected_rows(); + sqc_memstore_row_read_count += task.get_memstore_read_row_count(); + sqc_ssstore_row_read_count += task.get_ssstore_read_row_count(); finish_msg.dml_row_info_.add_px_dml_row_info(task.dml_row_info_); finish_msg.temp_table_id_ = task.temp_table_id_; if (OB_NOT_NULL(session)) { @@ -492,6 +496,8 @@ int ObPxSQCProxy::report(int end_ret) const sqc_ret = ret; } finish_msg.sqc_affected_rows_ = affected_rows; + finish_msg.sqc_memstore_row_read_count_ = sqc_memstore_row_read_count; + finish_msg.sqc_ssstore_row_read_count_ = sqc_ssstore_row_read_count; finish_msg.sqc_id_ = sqc.get_sqc_id(); finish_msg.dfo_id_ = sqc.get_dfo_id(); finish_msg.rc_ = sqc_ret; diff --git a/src/sql/engine/px/ob_px_task_process.cpp b/src/sql/engine/px/ob_px_task_process.cpp index 2b1c9380da..db57323384 100644 --- a/src/sql/engine/px/ob_px_task_process.cpp +++ b/src/sql/engine/px/ob_px_task_process.cpp @@ -201,6 +201,16 @@ int ObPxTaskProcess::process() audit_record.update_event_stage_state(); } + if (enable_sql_audit) { + if (OB_ISNULL(arg_.sqc_task_ptr_)){ + ret = OB_ERR_UNEXPECTED; + LOG_WARN("the sqc task ptr is null", K(ret)); + } else { + arg_.sqc_task_ptr_->set_memstore_read_row_count(exec_record.get_memstore_read_row_count()); + arg_.sqc_task_ptr_->set_ssstore_read_row_count(exec_record.get_ssstore_read_row_count()); + } + } + if (enable_sql_audit) { ObPhysicalPlan *phy_plan = arg_.des_phy_plan_; if ( OB_ISNULL(phy_plan)) { @@ -232,6 +242,8 @@ int ObPxTaskProcess::process() audit_record.is_hit_plan_cache_ = true; audit_record.is_multi_stmt_ = false; audit_record.is_perf_event_closed_ = !lib::is_diagnose_info_enabled(); + audit_record.total_memstore_read_row_count_ = exec_record.get_memstore_read_row_count(); + audit_record.total_ssstore_read_row_count_ = exec_record.get_ssstore_read_row_count(); } } ObSQLUtils::handle_audit_record(false, EXECUTE_DIST, *session); diff --git a/src/sql/engine/sequence/ob_sequence_op.cpp b/src/sql/engine/sequence/ob_sequence_op.cpp index 91d97a38be..e7e13355f1 100644 --- a/src/sql/engine/sequence/ob_sequence_op.cpp +++ b/src/sql/engine/sequence/ob_sequence_op.cpp @@ -143,11 +143,18 @@ int ObLocalSequenceExecutor::get_nextval(ObExecContext &ctx) ObSequenceValue seq_value; // 注意:这里 schema 的顺序和 ids 里面 id 的顺序是一一对应的 // 所以可以直接用下标来寻址 - if (OB_FAIL(sequence_cache_->nextval(seq_schemas_.at(idx), - allocator, - seq_value))) { - LOG_WARN("fail get nextval for seq", K(tenant_id), K(seq_id), K(ret)); - } else if (OB_FAIL(my_session->set_sequence_value(tenant_id, seq_id, seq_value))) { + ObAutoincrementService &auto_service = ObAutoincrementService::get_instance(); + if (seq_schemas_.at(idx).get_order_flag() + && seq_schemas_.at(idx).get_cache_order_mode() == NEW_ACTION) { + if (OB_FAIL(auto_service.get_handle(seq_schemas_.at(idx), seq_value))) { + LOG_WARN("fail get nextval from rpc for seq", K(tenant_id), K(seq_id), K(ret)); + } + } else { + if (OB_FAIL(sequence_cache_->nextval(seq_schemas_.at(idx), allocator, seq_value))) { + LOG_WARN("fail get nextval for seq", K(tenant_id), K(seq_id), K(ret)); + } + } + if (OB_SUCC(ret) && OB_FAIL(my_session->set_sequence_value(tenant_id, seq_id, seq_value))) { LOG_WARN("save seq_value to session as currval for later read fail", K(tenant_id), K(seq_id), K(seq_value), K(ret)); } diff --git a/src/sql/engine/sequence/ob_sequence_op.h b/src/sql/engine/sequence/ob_sequence_op.h index 94ac3a8631..c96ba82dd1 100644 --- a/src/sql/engine/sequence/ob_sequence_op.h +++ b/src/sql/engine/sequence/ob_sequence_op.h @@ -76,6 +76,8 @@ class ObLocalSequenceExecutor : public ObSequenceExecutor { virtual void reset() override; virtual void destroy() override; virtual int get_nextval(ObExecContext &ctx) override; + int handle_gais_request(const ObGAISNextSequenceValReq &request, + obrpc::ObGAISNextSequenceValRpcResult &result); private: // sequence 暴露给用户层的是一个 cache // cache 底层负责做 sequence 的缓存更新以及全局的协调 diff --git a/src/sql/executor/ob_direct_receive_op.cpp b/src/sql/executor/ob_direct_receive_op.cpp index 9ead7afd7f..8e23cf82a6 100644 --- a/src/sql/executor/ob_direct_receive_op.cpp +++ b/src/sql/executor/ob_direct_receive_op.cpp @@ -257,6 +257,8 @@ int ObDirectReceiveOp::setup_next_scanner() } } else { ret = OB_ITER_END; + plan_ctx->add_total_memstore_read_row_count(scanner_->get_memstore_read_row_count()); + plan_ctx->add_total_ssstore_read_row_count(scanner_->get_ssstore_read_row_count()); // only successful select affect last_insert_id // for select, last_insert_id may changed because last_insert_id(#) called // last_insert_id values should be the last row calling last_insert_id(#) diff --git a/src/sql/executor/ob_remote_executor_processor.cpp b/src/sql/executor/ob_remote_executor_processor.cpp index 882eeee77d..5003ae9cd0 100644 --- a/src/sql/executor/ob_remote_executor_processor.cpp +++ b/src/sql/executor/ob_remote_executor_processor.cpp @@ -358,6 +358,10 @@ int ObRemoteBaseExecuteP::sync_send_result(ObExecContext &exec_ctx, //scanner.set_force_rollback(plan_ctx->is_force_rollback()); } + scanner.set_memstore_read_row_count( + my_session->get_raw_audit_record().exec_record_.get_cur_memstore_read_row_count()); + scanner.set_ssstore_read_row_count( + my_session->get_raw_audit_record().exec_record_.get_cur_ssstore_read_row_count()); // set last_insert_id no matter success or fail after open scanner.set_last_insert_id_to_client(plan_ctx->calc_last_insert_id_to_client()); scanner.set_last_insert_id_session(plan_ctx->calc_last_insert_id_session()); diff --git a/src/sql/monitor/ob_exec_stat.h b/src/sql/monitor/ob_exec_stat.h index da64ecab0e..2676d37251 100644 --- a/src/sql/monitor/ob_exec_stat.h +++ b/src/sql/monitor/ob_exec_stat.h @@ -145,6 +145,32 @@ struct ObExecRecord UPDATE_EVENT(blockscan_row_cnt); UPDATE_EVENT(pushdown_storage_filter_row_cnt); } + + uint64_t get_cur_memstore_read_row_count(common::ObDiagnoseSessionInfo *di = NULL) { + oceanbase::common::ObDiagnoseSessionInfo *diag_session_info = + (NULL != di) ? di : oceanbase::common::ObDiagnoseSessionInfo::get_local_diagnose_info(); + uint64_t cur_memstore_read_row_count = 0; + if (NULL != diag_session_info) { + oceanbase::common::ObStatEventAddStatArray &arr = diag_session_info->get_add_stat_stats(); + cur_memstore_read_row_count = memstore_read_row_count_ + + (EVENT_STAT_GET(arr, ObStatEventIds::MEMSTORE_READ_ROW_COUNT) + - memstore_read_row_count_start_); + } + return cur_memstore_read_row_count; + } + + uint64_t get_cur_ssstore_read_row_count(common::ObDiagnoseSessionInfo *di = NULL) { + oceanbase::common::ObDiagnoseSessionInfo *diag_session_info = + (NULL != di) ? di : oceanbase::common::ObDiagnoseSessionInfo::get_local_diagnose_info(); + uint64_t cur_ssstore_read_row_count = 0; + if (NULL != diag_session_info) { + oceanbase::common::ObStatEventAddStatArray &arr = diag_session_info->get_add_stat_stats(); + cur_ssstore_read_row_count = ssstore_read_row_count_ + + (EVENT_STAT_GET(arr, ObStatEventIds::SSSTORE_READ_ROW_COUNT) + - ssstore_read_row_count_start_); + } + return cur_ssstore_read_row_count; + } }; enum ExecType { @@ -416,10 +442,10 @@ struct ObAuditRecordData { bool is_perf_event_closed_; char flt_trace_id_[OB_MAX_UUID_STR_LENGTH + 1]; char snapshot_source_[OB_MAX_SNAPSHOT_SOURCE_LENGTH + 1]; - uint64_t total_memstore_read_row_count_; - uint64_t total_ssstore_read_row_count_; ObCurTraceId::TraceId pl_trace_id_; int64_t plsql_exec_time_; + uint64_t total_memstore_read_row_count_; + uint64_t total_ssstore_read_row_count_; }; } //namespace sql diff --git a/src/sql/parser/non_reserved_keywords_mysql_mode.c b/src/sql/parser/non_reserved_keywords_mysql_mode.c index 456b3f110b..5866de330f 100644 --- a/src/sql/parser/non_reserved_keywords_mysql_mode.c +++ b/src/sql/parser/non_reserved_keywords_mysql_mode.c @@ -64,6 +64,7 @@ static const NonReservedKeyword Mysql_none_reserved_keywords[] = {"auto", AUTO}, {"auto_increment", AUTO_INCREMENT}, {"auto_increment_mode", AUTO_INCREMENT_MODE}, + {"auto_increment_cache_size", AUTO_INCREMENT_CACHE_SIZE}, {"auto_refresh", AUTO_REFRESH}, {"autoextend_size", AUTOEXTEND_SIZE}, {"availability", AVAILABILITY}, diff --git a/src/sql/parser/sql_parser_mysql_mode.y b/src/sql/parser/sql_parser_mysql_mode.y index 2711073b4f..2cbe37de3d 100644 --- a/src/sql/parser/sql_parser_mysql_mode.y +++ b/src/sql/parser/sql_parser_mysql_mode.y @@ -263,8 +263,8 @@ END_P SET_VAR DELIMITER //-----------------------------non_reserved keyword begin------------------------------------------- ACCESS ACCOUNT ACTION ACTIVE ADDDATE AFTER AGAINST AGGREGATE ALGORITHM ALL_META ALL_USER ALWAYS ALLOW ANALYSE ANY APPROX_COUNT_DISTINCT APPROX_COUNT_DISTINCT_SYNOPSIS APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE - ARBITRATION ARRAY ASCII ASIS AT AUTHORS AUTO AUTOEXTEND_SIZE AUTO_INCREMENT AUTO_INCREMENT_MODE AVG AVG_ROW_LENGTH - ACTIVATE AVAILABILITY ARCHIVELOG ASYNCHRONOUS AUDIT ADMIN AUTO_REFRESH + ARBITRATION ARRAY ASCII ASIS AT AUTHORS AUTO AUTOEXTEND_SIZE AUTO_INCREMENT AUTO_INCREMENT_MODE AUTO_INCREMENT_CACHE_SIZE + AVG AVG_ROW_LENGTH ACTIVATE AVAILABILITY ARCHIVELOG ASYNCHRONOUS AUDIT ADMIN AUTO_REFRESH BACKUP BACKUP_COPIES BALANCE BANDWIDTH BASE BASELINE BASELINE_ID BASIC BEGI BINDING SHARDING BINLOG BIT BIT_AND BIT_OR BIT_XOR BLOCK BLOCK_INDEX BLOCK_SIZE BLOOM_FILTER BOOL BOOLEAN BOOTSTRAP BTREE BYTE @@ -7072,6 +7072,11 @@ TABLE_MODE opt_equal_mark STRING_VALUE (void)($2); malloc_non_terminal_node($$, result->malloc_pool_, T_AUTO_INCREMENT_MODE, 1, $3); } +| AUTO_INCREMENT_CACHE_SIZE opt_equal_mark INTNUM +{ + (void)($2); + malloc_non_terminal_node($$, result->malloc_pool_, T_AUTO_INCREMENT_CACHE_SIZE, 1, $3); +} | ENABLE_EXTENDED_ROWID opt_equal_mark BOOL_VALUE { (void)($2); @@ -21582,6 +21587,7 @@ ACCOUNT | AUTOEXTEND_SIZE | AUTO_INCREMENT | AUTO_INCREMENT_MODE +| AUTO_INCREMENT_CACHE_SIZE | AUTO_REFRESH | AVG | AVG_ROW_LENGTH diff --git a/src/sql/resolver/ddl/ob_alter_sequence_resolver.cpp b/src/sql/resolver/ddl/ob_alter_sequence_resolver.cpp index 53a415e817..f9b7274118 100644 --- a/src/sql/resolver/ddl/ob_alter_sequence_resolver.cpp +++ b/src/sql/resolver/ddl/ob_alter_sequence_resolver.cpp @@ -111,7 +111,8 @@ int ObAlterSequenceResolver::resolve(const ParseNode &parse_tree) K(parse_tree.children_[1]->type_), K(ret)); } else { ObSequenceResolver resolver; - ret = resolver.resolve_sequence_options(mystmt, parse_tree.children_[1]); + ret = resolver.resolve_sequence_options(session_info_->get_effective_tenant_id(), mystmt, + parse_tree.children_[1]); } } else { ret = OB_ERR_REQUIRE_ALTER_SEQ_OPTION; diff --git a/src/sql/resolver/ddl/ob_alter_table_resolver.cpp b/src/sql/resolver/ddl/ob_alter_table_resolver.cpp index 91a9175116..98e1c7bb18 100644 --- a/src/sql/resolver/ddl/ob_alter_table_resolver.cpp +++ b/src/sql/resolver/ddl/ob_alter_table_resolver.cpp @@ -404,6 +404,7 @@ int ObAlterTableResolver::set_table_options() alter_table_schema.set_tablespace_id(tablespace_id_); alter_table_schema.set_dop(table_dop_); alter_table_schema.set_lob_inrow_threshold(lob_inrow_threshold_); + alter_table_schema.set_auto_increment_cache_size(auto_increment_cache_size_); //deep copy if (OB_FAIL(ret)) { //do nothing diff --git a/src/sql/resolver/ddl/ob_column_sequence_resolver.cpp b/src/sql/resolver/ddl/ob_column_sequence_resolver.cpp index 054fe37918..9cd2b8c526 100644 --- a/src/sql/resolver/ddl/ob_column_sequence_resolver.cpp +++ b/src/sql/resolver/ddl/ob_column_sequence_resolver.cpp @@ -79,7 +79,8 @@ int ObColumnSequenceResolver::resolve_sequence_without_name(ObColumnSequenceStmt LOG_WARN("invalid option node type", K(node->type_), K(ret)); } else { ObSequenceResolver resolver; - if (OB_FAIL(resolver.resolve_sequence_options(mystmt, node))) { + if (OB_FAIL(resolver.resolve_sequence_options(session_info_->get_effective_tenant_id(), + mystmt, node))) { LOG_WARN("resolve sequence options failed", K(ret)); } } diff --git a/src/sql/resolver/ddl/ob_create_sequence_resolver.cpp b/src/sql/resolver/ddl/ob_create_sequence_resolver.cpp index 1532767c27..965d216965 100644 --- a/src/sql/resolver/ddl/ob_create_sequence_resolver.cpp +++ b/src/sql/resolver/ddl/ob_create_sequence_resolver.cpp @@ -104,8 +104,6 @@ int ObCreateSequenceResolver::resolve(const ParseNode &parse_tree) } } - - /* sequence options */ if (OB_SUCC(ret) && NULL != parse_tree.children_[1]) { if (OB_UNLIKELY(T_SEQUENCE_OPTION_LIST != parse_tree.children_[1]->type_)) { @@ -114,7 +112,8 @@ int ObCreateSequenceResolver::resolve(const ParseNode &parse_tree) K(parse_tree.children_[1]->type_), K(ret)); } else { ObSequenceResolver resolver; - if (OB_FAIL(resolver.resolve_sequence_options(mystmt, parse_tree.children_[1]))) { + if (OB_FAIL(resolver.resolve_sequence_options(session_info_->get_effective_tenant_id(), + mystmt, parse_tree.children_[1]))) { LOG_WARN("resolve sequence options failed", K(ret)); } } diff --git a/src/sql/resolver/ddl/ob_create_table_resolver_base.cpp b/src/sql/resolver/ddl/ob_create_table_resolver_base.cpp index a9d6ec1c9a..f7d8fb64c0 100644 --- a/src/sql/resolver/ddl/ob_create_table_resolver_base.cpp +++ b/src/sql/resolver/ddl/ob_create_table_resolver_base.cpp @@ -404,6 +404,9 @@ int ObCreateTableResolverBase::set_table_option_to_schema(ObTableSchema &table_s LOG_USER_ERROR(OB_NOT_SUPPORTED, "Default format or location option for external table"); } } + if (OB_SUCC(ret) && auto_increment_cache_size_ != 0) { + table_schema.set_auto_increment_cache_size(auto_increment_cache_size_); + } } return ret; } diff --git a/src/sql/resolver/ddl/ob_ddl_resolver.cpp b/src/sql/resolver/ddl/ob_ddl_resolver.cpp index bc9233c21e..2a3e6eb229 100644 --- a/src/sql/resolver/ddl/ob_ddl_resolver.cpp +++ b/src/sql/resolver/ddl/ob_ddl_resolver.cpp @@ -117,7 +117,8 @@ ObDDLResolver::ObDDLResolver(ObResolverParams ¶ms) name_generated_type_(GENERATED_TYPE_UNKNOWN), have_generate_fts_arg_(false), is_set_lob_inrow_threshold_(false), - lob_inrow_threshold_(OB_DEFAULT_LOB_INROW_THRESHOLD) + lob_inrow_threshold_(OB_DEFAULT_LOB_INROW_THRESHOLD), + auto_increment_cache_size_(0) { table_mode_.reset(); } @@ -2029,6 +2030,45 @@ int ObDDLResolver::resolve_table_option(const ParseNode *option_node, const bool } break; } + case T_AUTO_INCREMENT_CACHE_SIZE: { + uint64_t tenant_data_version = 0; + if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) { + LOG_WARN("get tenant data version failed", K(ret)); + } else if (tenant_data_version < MOCK_DATA_VERSION_4_2_3_0 || + (tenant_data_version >= DATA_VERSION_4_3_0_0 && tenant_data_version < DATA_VERSION_4_3_2_0)) { + ret = OB_NOT_SUPPORTED; + LOG_WARN("alter table auto_increment_cache_size is not supported in data version less than 4.2.3", + K(ret), K(tenant_data_version)); + LOG_USER_ERROR(OB_NOT_SUPPORTED, "alter table auto_increment_cache_size is not supported in data version less than 4.2.3"); + } else if (OB_ISNULL(option_node->children_[0])) { + ret = OB_ERR_UNEXPECTED; + SQL_RESV_LOG(WARN, "option_node child is null", K(option_node->children_[0]), K(ret)); + } else { + const static int64_t MAX_AUTO_INCREMENT_CACHE_SIZE = 100000000; + const int64_t cache_size = option_node->children_[0]->value_; + if (cache_size < 0 || cache_size > MAX_AUTO_INCREMENT_CACHE_SIZE) { + ret = OB_INVALID_ARGUMENT; + SQL_RESV_LOG(WARN, "Specify table auto increment cache size should be [0, 100000000]", + K(ret), K(cache_size)); + LOG_USER_ERROR(OB_INVALID_ARGUMENT, "table auto_increment_cache_size"); + } else { + auto_increment_cache_size_ = cache_size; + } + } + if (OB_SUCCESS == ret && stmt::T_ALTER_TABLE ==stmt_->get_stmt_type()) { + HEAP_VAR(ObTableSchema, tmp_table_schema) { + if (OB_FAIL(get_table_schema_for_check(tmp_table_schema))) { + LOG_WARN("get table schema failed", K(ret)); + } else if (auto_increment_cache_size_ == + tmp_table_schema.get_auto_increment_cache_size()) { + // same as the original auto_increment_mode, do nothing + } else if (OB_FAIL(alter_table_bitset_.add_member(ObAlterTableArg::INCREMENT_CACHE_SIZE))) { + SQL_RESV_LOG(WARN, "failed to add member to bitset!", K(ret)); + } + } + } + break; + } case T_ENABLE_EXTENDED_ROWID: { if (OB_ISNULL(option_node->children_[0])) { ret = OB_ERR_UNEXPECTED; @@ -2445,7 +2485,7 @@ int ObDDLResolver::resolve_table_option(const ParseNode *option_node, const bool break; } case T_TTL_DEFINITION: { - uint64_t tenant_data_version = 0;; + uint64_t tenant_data_version = 0; if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) { LOG_WARN("get tenant data version failed", K(ret)); } else if (tenant_data_version < DATA_VERSION_4_2_1_0) { @@ -2481,7 +2521,7 @@ int ObDDLResolver::resolve_table_option(const ParseNode *option_node, const bool break; } case T_KV_ATTRIBUTES: { - uint64_t tenant_data_version = 0;; + uint64_t tenant_data_version = 0; if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) { LOG_WARN("get tenant data version failed", K(ret)); } else if (tenant_data_version < DATA_VERSION_4_2_1_0) { @@ -4129,7 +4169,7 @@ int ObDDLResolver::resolve_srid_node(share::schema::ObColumnSchemaV2 &column, { int ret = OB_SUCCESS; uint64_t tenant_id = session_info_->get_effective_tenant_id(); - uint64_t tenant_data_version = 0;; + uint64_t tenant_data_version = 0; if (OB_FAIL(GET_MIN_DATA_VERSION(tenant_id, tenant_data_version))) { LOG_WARN("get tenant data version failed", K(ret)); @@ -4369,7 +4409,7 @@ int ObDDLResolver::resolve_lob_inrow_threshold(const ParseNode *option_node, con { int ret = OB_SUCCESS; uint64_t tenant_id = 0; - uint64_t tenant_data_version = 0;; + uint64_t tenant_data_version = 0; if (OB_ISNULL(session_info_)) { ret = OB_ERR_UNEXPECTED; SQL_RESV_LOG(WARN, "session_info_ is null", K(ret)); @@ -4975,6 +5015,7 @@ void ObDDLResolver::reset() { kv_attributes_.reset(); is_set_lob_inrow_threshold_ = false; lob_inrow_threshold_ = OB_DEFAULT_LOB_INROW_THRESHOLD; + auto_increment_cache_size_ = 0; } bool ObDDLResolver::is_valid_prefix_key_type(const ObObjTypeClass column_type_class) diff --git a/src/sql/resolver/ddl/ob_ddl_resolver.h b/src/sql/resolver/ddl/ob_ddl_resolver.h index 5135a953dd..f1f6da945d 100644 --- a/src/sql/resolver/ddl/ob_ddl_resolver.h +++ b/src/sql/resolver/ddl/ob_ddl_resolver.h @@ -1027,6 +1027,7 @@ protected: bool have_generate_fts_arg_; bool is_set_lob_inrow_threshold_; int64_t lob_inrow_threshold_; + int64_t auto_increment_cache_size_; private: template DISALLOW_COPY_AND_ASSIGN(ObDDLResolver); diff --git a/src/sql/resolver/ddl/ob_sequence_resolver.h b/src/sql/resolver/ddl/ob_sequence_resolver.h index 89da2e18dc..a1c4e2b9b9 100644 --- a/src/sql/resolver/ddl/ob_sequence_resolver.h +++ b/src/sql/resolver/ddl/ob_sequence_resolver.h @@ -41,7 +41,7 @@ public: } ~ObSequenceResolver() = default; public: - int resolve_sequence_options(T *stmt, ParseNode *node); + int resolve_sequence_options(uint64_t tenant_id, T *stmt, ParseNode *node); private: int resolve_sequence_option(T *stmt, ParseNode *node); int resolve_sequence_option_inner(ParseNode &option_node, @@ -56,7 +56,7 @@ private: }; template -int ObSequenceResolver::resolve_sequence_options(T *stmt, ParseNode *node) +int ObSequenceResolver::resolve_sequence_options(uint64_t tenant_id, T *stmt, ParseNode *node) { int ret = common::OB_SUCCESS; if (OB_LIKELY(node)) { @@ -74,6 +74,15 @@ int ObSequenceResolver::resolve_sequence_options(T *stmt, ParseNode *node) SQL_LOG(WARN, "resolve sequence option failed", K(ret)); } } + // Fields used for upgrade compatibility, not user input + uint64_t compat_version = 0; + if (FAILEDx(GET_MIN_DATA_VERSION(tenant_id, compat_version))) { + LOG_WARN("fail to get data version", KR(ret), K(tenant_id)); + } else if ((compat_version >= MOCK_DATA_VERSION_4_2_3_0 + && compat_version < DATA_VERSION_4_3_0_0) + || (compat_version >= DATA_VERSION_4_3_2_0)) { + stmt->option().set_cache_order_mode(NEW_ACTION); + } // conflict check if (OB_SUCC(ret)) { diff --git a/src/sql/resolver/dml/ob_del_upd_resolver.cpp b/src/sql/resolver/dml/ob_del_upd_resolver.cpp index a7390ea682..1a53616a6c 100644 --- a/src/sql/resolver/dml/ob_del_upd_resolver.cpp +++ b/src/sql/resolver/dml/ob_del_upd_resolver.cpp @@ -3149,7 +3149,8 @@ int ObDelUpdResolver::build_autoinc_param( param.autoinc_first_part_num_ = table_schema->get_first_part_num(); param.autoinc_table_part_num_ = table_schema->get_all_part_num(); param.autoinc_col_id_ = column_id; - param.auto_increment_cache_size_ = auto_increment_cache_size; + param.auto_increment_cache_size_ = get_auto_increment_cache_size( + table_schema->get_auto_increment_cache_size(), auto_increment_cache_size); param.part_level_ = table_schema->get_part_level(); param.autoinc_col_type_ = column_type; param.autoinc_desired_count_ = 0; diff --git a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/desc_sys_views_in_mysql.result b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/desc_sys_views_in_mysql.result index ebea236ac5..d422706e3c 100644 --- a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/desc_sys_views_in_mysql.result +++ b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/desc_sys_views_in_mysql.result @@ -1103,10 +1103,10 @@ PHYSICAL_READ_REQUESTS bigint(0) unsigned NO PHYSICAL_READ_BYTES bigint(0) unsigned NO PHYSICAL_WRITE_REQUESTS bigint(0) unsigned NO PHYSICAL_WRITE_BYTES bigint(0) unsigned NO -WORKAREA_MEM bigint(0) unsigned NO -WORKAREA_MAX_MEM bigint(0) unsigned NO -WORKAREA_TEMPSEG bigint(0) unsigned NO -WORKAREA_MAX_TEMPSEG bigint(0) unsigned NO +WORKAREA_MEM bigint(20) unsigned NO NULL +WORKAREA_MAX_MEM bigint(20) unsigned NO NULL +WORKAREA_TEMPSEG bigint(20) unsigned NO NULL +WORKAREA_MAX_TEMPSEG bigint(20) unsigned NO NULL OTHERSTAT_GROUP_ID bigint(0) unsigned NO OTHERSTAT_1_ID bigint(20) NO NULL OTHERSTAT_1_TYPE bigint(0) unsigned NO @@ -1194,10 +1194,10 @@ PHYSICAL_READ_REQUESTS bigint(0) unsigned NO PHYSICAL_READ_BYTES bigint(0) unsigned NO PHYSICAL_WRITE_REQUESTS bigint(0) unsigned NO PHYSICAL_WRITE_BYTES bigint(0) unsigned NO -WORKAREA_MEM bigint(0) unsigned NO -WORKAREA_MAX_MEM bigint(0) unsigned NO -WORKAREA_TEMPSEG bigint(0) unsigned NO -WORKAREA_MAX_TEMPSEG bigint(0) unsigned NO +WORKAREA_MEM bigint(20) unsigned NO +WORKAREA_MAX_MEM bigint(20) unsigned NO +WORKAREA_TEMPSEG bigint(20) unsigned NO +WORKAREA_MAX_TEMPSEG bigint(20) unsigned NO OTHERSTAT_GROUP_ID bigint(0) unsigned NO OTHERSTAT_1_ID bigint(20) NO OTHERSTAT_1_TYPE bigint(0) unsigned NO diff --git a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/desc_sys_views_in_sys.result b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/desc_sys_views_in_sys.result index f89a02ef12..a0cc3e7ed0 100644 --- a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/desc_sys_views_in_sys.result +++ b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/desc_sys_views_in_sys.result @@ -1104,10 +1104,10 @@ PHYSICAL_READ_REQUESTS bigint(0) unsigned NO PHYSICAL_READ_BYTES bigint(0) unsigned NO PHYSICAL_WRITE_REQUESTS bigint(0) unsigned NO PHYSICAL_WRITE_BYTES bigint(0) unsigned NO -WORKAREA_MEM bigint(0) unsigned NO -WORKAREA_MAX_MEM bigint(0) unsigned NO -WORKAREA_TEMPSEG bigint(0) unsigned NO -WORKAREA_MAX_TEMPSEG bigint(0) unsigned NO +WORKAREA_MEM bigint(20) unsigned NO NULL +WORKAREA_MAX_MEM bigint(20) unsigned NO NULL +WORKAREA_TEMPSEG bigint(20) unsigned NO NULL +WORKAREA_MAX_TEMPSEG bigint(20) unsigned NO NULL OTHERSTAT_GROUP_ID bigint(0) unsigned NO OTHERSTAT_1_ID bigint(20) NO NULL OTHERSTAT_1_TYPE bigint(0) unsigned NO @@ -1195,10 +1195,10 @@ PHYSICAL_READ_REQUESTS bigint(0) unsigned NO PHYSICAL_READ_BYTES bigint(0) unsigned NO PHYSICAL_WRITE_REQUESTS bigint(0) unsigned NO PHYSICAL_WRITE_BYTES bigint(0) unsigned NO -WORKAREA_MEM bigint(0) unsigned NO -WORKAREA_MAX_MEM bigint(0) unsigned NO -WORKAREA_TEMPSEG bigint(0) unsigned NO -WORKAREA_MAX_TEMPSEG bigint(0) unsigned NO +WORKAREA_MEM bigint(20) unsigned NO +WORKAREA_MAX_MEM bigint(20) unsigned NO +WORKAREA_TEMPSEG bigint(20) unsigned NO +WORKAREA_MAX_TEMPSEG bigint(20) unsigned NO OTHERSTAT_GROUP_ID bigint(0) unsigned NO OTHERSTAT_1_ID bigint(20) NO OTHERSTAT_1_TYPE bigint(0) unsigned NO