patch 4.0
This commit is contained in:
@ -13,7 +13,7 @@
|
||||
#define USING_LOG_PREFIX SQL_EXE
|
||||
|
||||
#include "ob_granule_util.h"
|
||||
#include "share/ob_i_data_access_service.h"
|
||||
#include "share/ob_i_tablet_scan.h"
|
||||
#include "share/config/ob_server_config.h"
|
||||
#include "lib/ob_errno.h"
|
||||
#include "sql/ob_sql_define.h"
|
||||
@ -21,10 +21,15 @@
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/engine/px/ob_px_util.h"
|
||||
#include "ob_granule_pump.h"
|
||||
#include "storage/tx_storage/ob_access_service.h"
|
||||
#include "share/schema/ob_table_param.h"
|
||||
#include "sql/engine/ob_engine_op_traits.h"
|
||||
|
||||
using namespace oceanbase::common;
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace sql
|
||||
{
|
||||
|
||||
void ObParallelBlockRangeTaskParams::reset()
|
||||
{
|
||||
@ -38,71 +43,22 @@ void ObParallelBlockRangeTaskParams::reset()
|
||||
int ObParallelBlockRangeTaskParams::valid() const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (min_task_count_per_thread_ <= 0 || max_task_count_per_thread_ <= 0 || min_task_access_size_ <= 0 ||
|
||||
parallelism_ <= 0 || expected_task_load_ <= 0) {
|
||||
if (min_task_count_per_thread_ <= 0
|
||||
|| max_task_count_per_thread_ <= 0
|
||||
|| min_task_access_size_ <= 0
|
||||
|| parallelism_ <= 0
|
||||
|| expected_task_load_ <= 0) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("params is invalid", K(*this), K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGranuleUtil::compute_task_count(
|
||||
const ObParallelBlockRangeTaskParams& params, uint64_t marcos_count, int64_t& tasks_count)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t tmp_total_task_count = -1;
|
||||
// int64_t marcos_count = params.marcos_count_;
|
||||
int64_t macro_block_size = (OB_DEFAULT_MACRO_BLOCK_SIZE) >> 20; // macro block size (MB)
|
||||
if (OB_FAIL(params.valid()) || macro_block_size <= 0) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("params is invalid", K(OB_DEFAULT_MACRO_BLOCK_SIZE), K(ret));
|
||||
} else if (marcos_count == 0) {
|
||||
tmp_total_task_count = 1;
|
||||
} else {
|
||||
int64_t total_access_size = marcos_count * macro_block_size;
|
||||
// The minimum amount of data that must be read by a single task. The default is obtained
|
||||
// from the system configuration item, and the default is 2M
|
||||
int64_t min_task_access_size = NON_ZERO_VALUE(params.min_task_access_size_); //(MB)
|
||||
int64_t expected_task_load = max(params.expected_task_load_, min_task_access_size);
|
||||
int64_t user_expect_task_count_ = NON_ZERO_VALUE(total_access_size / min_task_access_size);
|
||||
/**
|
||||
* The min_task_count_per_thread_ defaults to 13, which means that a thread is expected to have at least 13 tasks
|
||||
* lower_bound_size = parallelism * 100M * 13
|
||||
* When total_access_size is less than this value, you must ensure that min_task_access_size
|
||||
* is less than or equal to the amount of data read by a single task
|
||||
*/
|
||||
int64_t lower_bound_size = params.parallelism_ * expected_task_load * params.min_task_count_per_thread_;
|
||||
/**
|
||||
* max_task_count_per_thread_ defaults to 100, which means that one thread is expected to have at most 100 tasks
|
||||
* upper_bound_size = parallelism * 100M * 100
|
||||
* When total_access_size is greater than this value, we expand the amount of data processed by each task
|
||||
*/
|
||||
int64_t upper_bound_size = params.parallelism_ * expected_task_load * params.max_task_count_per_thread_;
|
||||
if (total_access_size < 0 || lower_bound_size < 0 || upper_bound_size < 0 || min_task_access_size <= 0) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("params is invalid", K(total_access_size), K(lower_bound_size), K(upper_bound_size), K(params));
|
||||
} else if (total_access_size < lower_bound_size) {
|
||||
// Try to divide it according to 13 tasks in a thread, but this task cannot be less
|
||||
// than micro-blocks, because a task contains at least one micro-block
|
||||
tmp_total_task_count = min(params.min_task_count_per_thread_ * params.parallelism_, marcos_count);
|
||||
// Ensure that the minimum amount of data that a single task must read is min_task_access_size
|
||||
tmp_total_task_count = min(tmp_total_task_count, user_expect_task_count_);
|
||||
} else if (total_access_size > upper_bound_size) {
|
||||
// Expand the amount of data processed by each task
|
||||
tmp_total_task_count = params.max_task_count_per_thread_ * params.parallelism_;
|
||||
} else {
|
||||
// Each task reads expected_task_load size data, the number of tasks per thread is between [13, 100]
|
||||
tmp_total_task_count = total_access_size / expected_task_load;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
tasks_count = tmp_total_task_count;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObGranuleUtil::is_partition_granule(int64_t partition_count, int64_t parallelism, int64_t partition_scan_hold,
|
||||
int64_t hash_partition_scan_hold, bool hash_part)
|
||||
bool ObGranuleUtil::is_partition_granule(int64_t partition_count,
|
||||
int64_t parallelism,
|
||||
int64_t partition_scan_hold,
|
||||
int64_t hash_partition_scan_hold,
|
||||
bool hash_part)
|
||||
{
|
||||
bool partition_granule = false;
|
||||
// if parallelism is too small, we use partition granule.
|
||||
@ -114,94 +70,86 @@ bool ObGranuleUtil::is_partition_granule(int64_t partition_count, int64_t parall
|
||||
return partition_granule;
|
||||
}
|
||||
|
||||
int ObGranuleUtil::split_block_ranges(ObIAllocator& allocator, const ObIArray<common::ObNewRange>& in_ranges,
|
||||
const ObIArray<ObPartitionKey>& pkeys, storage::ObPartitionService& partition_service, int64_t parallelism,
|
||||
int64_t tablet_size, bool force_partition_granule, common::ObIArray<common::ObNewRange>& granule_ranges,
|
||||
common::ObIArray<int64_t>& offsets, common::ObIArray<int64_t>& partition_offsets)
|
||||
int ObGranuleUtil::split_block_ranges(ObIAllocator &allocator,
|
||||
const ObTableScanSpec *tsc,//may be is null, attention use
|
||||
const ObIArray<common::ObNewRange> &in_ranges,
|
||||
const ObIArray<ObDASTabletLoc*> &tablets,
|
||||
int64_t parallelism,
|
||||
int64_t tablet_size,
|
||||
bool force_partition_granule,
|
||||
common::ObIArray<ObDASTabletLoc*> &granule_tablets,
|
||||
common::ObIArray<common::ObNewRange> &granule_ranges,
|
||||
common::ObIArray<int64_t> &granule_idx,
|
||||
bool range_independent)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t total_macros_count = 0;
|
||||
int64_t total_task_count = 1;
|
||||
int64_t macros_count_per_task = 0;
|
||||
int64_t empty_partition_count = 0;
|
||||
common::ObSEArray<uint64_t, 16> macros_count_by_partition;
|
||||
common::ObSEArray<int64_t, 16> macros_count_by_partition_int64;
|
||||
common::ObSEArray<int64_t, 16> task_count_by_partition;
|
||||
common::ObSEArray<common::ObNewRange, 16> ranges;
|
||||
bool only_false_range = false;
|
||||
bool only_empty_range = false;
|
||||
|
||||
/**
|
||||
* prepare
|
||||
*/
|
||||
if (in_ranges.count() <= 0 || pkeys.count() <= 0) {
|
||||
if (in_ranges.count() <= 0 || tablets.count() <= 0) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("ranges/pkeys is empty", K(in_ranges), K(pkeys), K(ret));
|
||||
} else if (OB_FAIL(remove_false_range(in_ranges, ranges, only_false_range))) {
|
||||
LOG_WARN("failed to remove false range", K(ret));
|
||||
} else if (force_partition_granule || only_false_range) {
|
||||
LOG_WARN("ranges/tablets is empty", K(in_ranges), K(tablets), K(ret));
|
||||
} else if (OB_FAIL(remove_empty_range(in_ranges, ranges, only_empty_range))) {
|
||||
LOG_WARN("failed to remove empty range", K(ret));
|
||||
} else if (force_partition_granule
|
||||
|| only_empty_range) {
|
||||
// partition granule iterator
|
||||
// in the case of splitting tasks according to partition granularity, the number of tasks
|
||||
// is equal to the number of partitions (`pkeys.count()`)
|
||||
FOREACH_CNT_X(pkey, pkeys, OB_SUCC(ret))
|
||||
{
|
||||
UNUSED(pkey);
|
||||
FOREACH_CNT_X(range, ranges, OB_SUCC(ret))
|
||||
{
|
||||
if (OB_FAIL(granule_ranges.push_back(*range))) {
|
||||
// 按照partition粒度切分任务的情况下,任务的个数等于partition的个数(`tablets.count()`)
|
||||
int64_t pk_idx = 0;
|
||||
FOREACH_CNT_X(tablet, tablets, OB_SUCC(ret)) {
|
||||
FOREACH_CNT_X(range, ranges, OB_SUCC(ret)) {
|
||||
if (OB_FAIL(granule_tablets.push_back(*tablet))) {
|
||||
LOG_WARN("push basck tablet failed", K(ret));
|
||||
} else if (OB_FAIL(granule_ranges.push_back(*range))) {
|
||||
LOG_WARN("push back range failed", K(ret));
|
||||
} else if (OB_FAIL(granule_idx.push_back(pk_idx))) {
|
||||
LOG_WARN("push back pk_idx failed", K(ret));
|
||||
} else if (range_independent) {
|
||||
pk_idx++;
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(offsets.push_back(granule_ranges.count() - 1))) {
|
||||
LOG_WARN("push back range failed", K(ret));
|
||||
if (!range_independent) {
|
||||
pk_idx++;
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(partition_offsets.assign(offsets))) {
|
||||
LOG_WARN("assign partition offset failed", K(ret));
|
||||
}
|
||||
LOG_TRACE("gi partition granule");
|
||||
} else if (OB_FAIL(split_block_granule(allocator,
|
||||
in_ranges,
|
||||
pkeys,
|
||||
partition_service,
|
||||
parallelism,
|
||||
tablet_size,
|
||||
granule_ranges,
|
||||
offsets,
|
||||
partition_offsets))) {
|
||||
tsc,
|
||||
ranges,
|
||||
tablets,
|
||||
parallelism,
|
||||
tablet_size,
|
||||
granule_tablets,
|
||||
granule_ranges,
|
||||
granule_idx,
|
||||
range_independent))) {
|
||||
LOG_WARN("failed to split block granule tasks", K(ret));
|
||||
} else {
|
||||
LOG_TRACE("get the splited results through the new gi split method",
|
||||
K(ret),
|
||||
K(granule_ranges.count()),
|
||||
K(offsets.count()),
|
||||
K(offsets));
|
||||
K(ret), K(granule_tablets.count()), K(granule_ranges.count()), K(granule_idx));
|
||||
}
|
||||
LOG_TRACE("split ranges to granule",
|
||||
K(ret),
|
||||
K(total_task_count),
|
||||
K(parallelism),
|
||||
K(total_macros_count),
|
||||
K(macros_count_by_partition),
|
||||
K(macros_count_per_task),
|
||||
K(granule_ranges.count()),
|
||||
K(granule_ranges),
|
||||
K(offsets.count()),
|
||||
K(offsets),
|
||||
K(partition_offsets.count()),
|
||||
K(partition_offsets),
|
||||
K(pkeys),
|
||||
K(task_count_by_partition));
|
||||
LOG_TRACE("split ranges to granule", K(ret), K(total_task_count), K(parallelism),
|
||||
K(total_macros_count), K(macros_count_by_partition), K(macros_count_per_task),
|
||||
K(granule_tablets.count()), K(granule_tablets), K(granule_ranges.count()), K(granule_ranges),
|
||||
K(granule_idx.count()), K(granule_idx), K(tablets), K(task_count_by_partition));
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGranuleUtil::remove_false_range(const common::ObIArray<common::ObNewRange>& in_ranges,
|
||||
common::ObIArray<common::ObNewRange>& ranges, bool& only_false_range)
|
||||
{
|
||||
int ObGranuleUtil::remove_empty_range(const common::ObIArray<common::ObNewRange> &in_ranges,
|
||||
common::ObIArray<common::ObNewRange> &ranges,
|
||||
bool &only_empty_range) {
|
||||
int ret = OB_SUCCESS;
|
||||
for (int64_t i = 0; i < in_ranges.count() && OB_SUCC(ret); ++i) {
|
||||
if (!in_ranges.at(i).is_false_range()) {
|
||||
if (!in_ranges.at(i).empty()) {
|
||||
if (OB_FAIL(ranges.push_back(in_ranges.at(i)))) {
|
||||
LOG_WARN("fail to push back ranges", K(ret));
|
||||
}
|
||||
@ -211,18 +159,23 @@ int ObGranuleUtil::remove_false_range(const common::ObIArray<common::ObNewRange>
|
||||
if (OB_FAIL(ranges.assign(in_ranges))) {
|
||||
LOG_WARN("failed to assign ranges", K(ret));
|
||||
} else {
|
||||
only_false_range = true;
|
||||
only_empty_range = true;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGranuleUtil::split_block_granule(ObIAllocator& allocator, const ObIArray<common::ObNewRange>& input_ranges,
|
||||
const ObIArray<ObPartitionKey>& pkeys, storage::ObPartitionService& partition_service, int64_t parallelism,
|
||||
int64_t tablet_size, common::ObIArray<common::ObNewRange>& tasks_ranges, common::ObIArray<int64_t>& tasks_offsets,
|
||||
common::ObIArray<int64_t>& tasks_partition_offsets)
|
||||
int ObGranuleUtil::split_block_granule(ObIAllocator &allocator,
|
||||
const ObTableScanSpec *tsc,//may be is null, attention use!
|
||||
const ObIArray<common::ObNewRange> &input_ranges,
|
||||
const ObIArray<ObDASTabletLoc*> &tablets,
|
||||
int64_t parallelism,
|
||||
int64_t tablet_size,
|
||||
common::ObIArray<ObDASTabletLoc*> &granule_tablets,
|
||||
common::ObIArray<common::ObNewRange> &granule_ranges,
|
||||
common::ObIArray<int64_t> &granule_idx,
|
||||
bool range_independent)
|
||||
{
|
||||
|
||||
// the step for split task by block granule method:
|
||||
// 1. check the validity of input parameters
|
||||
// 2. get size for each partition, and calc the total size for all partitions
|
||||
@ -231,10 +184,12 @@ int ObGranuleUtil::split_block_granule(ObIAllocator& allocator, const ObIArray<c
|
||||
// 5. calculate task ranges for each partition, and get the result
|
||||
|
||||
int ret = OB_SUCCESS;
|
||||
ObAccessService *access_service = MTL(ObAccessService *);
|
||||
// 1. check the validity of input parameters
|
||||
if (input_ranges.count() < 1 || pkeys.count() < 1 || parallelism < 1 || tablet_size < 1) {
|
||||
if (input_ranges.count() < 1 || tablets.count() < 1 || parallelism < 1 || tablet_size < 1) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("the invalid argument", K(ret), K(input_ranges.count()), K(pkeys.count()), K(parallelism), K(tablet_size));
|
||||
LOG_WARN("the invalid argument",
|
||||
K(ret), K(input_ranges.count()), K(tablets.count()), K(parallelism), K(tablet_size));
|
||||
}
|
||||
|
||||
// 2. get size for each partition, and calc the total size for all partitions
|
||||
@ -242,21 +197,25 @@ int ObGranuleUtil::split_block_granule(ObIAllocator& allocator, const ObIArray<c
|
||||
int64_t total_size = 0;
|
||||
int64_t empty_partition_cnt = 0;
|
||||
ObSEArray<ObStoreRange, 16> input_store_ranges;
|
||||
bool need_convert_new_range = true;//only rowid range need extra convert.
|
||||
if (OB_SUCC(ret)) {
|
||||
// convert ObNewRange array to ObStoreRange array
|
||||
ObStoreRange store_range;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < input_ranges.count(); i++) {
|
||||
store_range.assign(input_ranges.at(i));
|
||||
if (OB_FAIL(input_store_ranges.push_back(store_range))) {
|
||||
LOG_WARN("failed to push back input store range", K(ret));
|
||||
}
|
||||
}
|
||||
for (int i = 0; i < pkeys.count() && OB_SUCC(ret); i++) {
|
||||
ObPartitionKey partition_key = pkeys.at(i);
|
||||
for (int i = 0; i < tablets.count() && OB_SUCC(ret); i++) {
|
||||
const ObDASTabletLoc &tablet = *tablets.at(i);
|
||||
int64_t partition_size = 0;
|
||||
// get partition size from storage
|
||||
if (OB_FAIL(partition_service.get_multi_ranges_cost(partition_key, input_store_ranges, partition_size))) {
|
||||
LOG_WARN("failed to get multi ranges cost", K(ret), K(partition_key));
|
||||
if (need_convert_new_range &&
|
||||
OB_FAIL(convert_new_range_to_store_range(allocator,
|
||||
tsc,
|
||||
tablet.tablet_id_,
|
||||
input_ranges,
|
||||
input_store_ranges,
|
||||
need_convert_new_range))) {
|
||||
LOG_WARN("failed to convert new range to store range", K(ret));
|
||||
} else if (OB_FAIL(access_service->get_multi_ranges_cost(tablet.ls_id_,
|
||||
tablet.tablet_id_,
|
||||
input_store_ranges,
|
||||
partition_size))) {
|
||||
LOG_WARN("failed to get multi ranges cost", K(ret), K(tablet));
|
||||
} else {
|
||||
// B to MB
|
||||
partition_size = partition_size / 1024 / 1024;
|
||||
@ -280,14 +239,14 @@ int ObGranuleUtil::split_block_granule(ObIAllocator& allocator, const ObIArray<c
|
||||
if (OB_SUCC(ret)) {
|
||||
ObParallelBlockRangeTaskParams params;
|
||||
params.parallelism_ = parallelism;
|
||||
params.expected_task_load_ = tablet_size / 1024 / 1024;
|
||||
params.expected_task_load_ = tablet_size/1024/1024;
|
||||
if (OB_FAIL(compute_total_task_count(params, total_size, esti_task_cnt_by_data_size))) {
|
||||
LOG_WARN("compute task count failed", K(ret));
|
||||
} else {
|
||||
esti_task_cnt_by_data_size += empty_partition_cnt;
|
||||
// Ensure that the total task count is greater than or equal to the number of partitions
|
||||
if (esti_task_cnt_by_data_size < pkeys.count()) {
|
||||
esti_task_cnt_by_data_size = pkeys.count();
|
||||
// 确保total task count是大于等于partition的个数的
|
||||
if (esti_task_cnt_by_data_size < tablets.count()) {
|
||||
esti_task_cnt_by_data_size = tablets.count();
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -295,54 +254,65 @@ int ObGranuleUtil::split_block_granule(ObIAllocator& allocator, const ObIArray<c
|
||||
// 4. split the total number of tasks into each partition
|
||||
common::ObSEArray<int64_t, 16> task_cnt_each_partitions;
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(compute_task_count_each_partition(
|
||||
total_size, esti_task_cnt_by_data_size, size_each_partitions, task_cnt_each_partitions))) {
|
||||
if (OB_FAIL(compute_task_count_each_partition(total_size,
|
||||
esti_task_cnt_by_data_size,
|
||||
size_each_partitions,
|
||||
task_cnt_each_partitions))) {
|
||||
LOG_WARN("failed to compute task count for each partition", K(ret));
|
||||
}
|
||||
}
|
||||
|
||||
// 5. calc task ranges for each partition, and get the result
|
||||
if (OB_SUCC(ret)) {
|
||||
for (int i = 0; i < pkeys.count() && OB_SUCC(ret); i++) {
|
||||
ObPartitionKey partition_key = pkeys.at(i);
|
||||
int64_t tablet_idx = 0;
|
||||
for (int i = 0; i < tablets.count() && OB_SUCC(ret); i++) {
|
||||
ObDASTabletLoc *tablet = tablets.at(i);
|
||||
int64_t expected_task_cnt = task_cnt_each_partitions.at(i);
|
||||
// split input ranges to n task by PG interface
|
||||
if (OB_FAIL(get_tasks_for_partition(allocator,
|
||||
expected_task_cnt,
|
||||
partition_key,
|
||||
partition_service,
|
||||
input_store_ranges,
|
||||
tasks_ranges,
|
||||
tasks_offsets,
|
||||
tasks_partition_offsets))) {
|
||||
if (need_convert_new_range &&
|
||||
OB_FAIL(convert_new_range_to_store_range(allocator,
|
||||
tsc,
|
||||
tablet->tablet_id_,
|
||||
input_ranges,
|
||||
input_store_ranges,
|
||||
need_convert_new_range))) {
|
||||
LOG_WARN("failed to convert new range to store range", K(ret));
|
||||
} else if (OB_FAIL(get_tasks_for_partition(allocator,
|
||||
expected_task_cnt,
|
||||
*tablet,
|
||||
input_store_ranges,
|
||||
granule_tablets,
|
||||
granule_ranges,
|
||||
granule_idx,
|
||||
tablet_idx,
|
||||
range_independent))) {
|
||||
LOG_WARN("failed to get tasks for partition", K(ret));
|
||||
} else {
|
||||
LOG_TRACE("get tasks for partition",
|
||||
K(ret),
|
||||
K(partition_key),
|
||||
K(tasks_ranges.count()),
|
||||
K(tasks_offsets),
|
||||
K(tasks_partition_offsets));
|
||||
K(ret), KPC(tablet), K(granule_ranges.count()), K(granule_tablets), K(granule_idx));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (tasks_ranges.empty() || tasks_offsets.empty() || tasks_partition_offsets.empty()) {
|
||||
if (granule_tablets.empty() ||
|
||||
granule_tablets.count() != granule_ranges.count() ||
|
||||
granule_tablets.count() != granule_idx.count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("the ranges or offsets are empty", K(ret), K(tasks_ranges.empty()), K(tasks_offsets.empty()));
|
||||
LOG_WARN("the ranges or offsets are empty",
|
||||
K(ret), K(granule_tablets.count()), K(granule_ranges.count()), K(granule_idx.count()));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGranuleUtil::compute_total_task_count(
|
||||
const ObParallelBlockRangeTaskParams& params, int64_t total_size, int64_t& total_task_count)
|
||||
int ObGranuleUtil::compute_total_task_count(const ObParallelBlockRangeTaskParams ¶ms,
|
||||
int64_t total_size,
|
||||
int64_t &total_task_count)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t tmp_total_task_count = -1;
|
||||
if (OB_FAIL(params.valid())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("params is invalid", K(ret));
|
||||
LOG_WARN("params is invalid" , K(ret));
|
||||
} else {
|
||||
// total size
|
||||
int64_t total_access_size = total_size;
|
||||
@ -356,15 +326,16 @@ int ObGranuleUtil::compute_total_task_count(
|
||||
// hight bound size: dop*128M*100
|
||||
int64_t upper_bound_size = params.parallelism_ * expected_task_load * params.max_task_count_per_thread_;
|
||||
|
||||
if (total_access_size < 0 || lower_bound_size < 0 || upper_bound_size < 0) {
|
||||
if (total_access_size < 0 || lower_bound_size < 0 || upper_bound_size < 0 ) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("params is invalid", K(total_access_size), K(lower_bound_size), K(upper_bound_size), K(params));
|
||||
LOG_WARN("params is invalid",
|
||||
K(total_access_size), K(lower_bound_size), K(upper_bound_size), K(params));
|
||||
} else if (total_access_size < lower_bound_size) {
|
||||
// the data size is less than lower bound size
|
||||
// when the amount of data is small,
|
||||
// more tasks can easily achieve better dynamic load balancing
|
||||
tmp_total_task_count =
|
||||
min(params.min_task_count_per_thread_ * params.parallelism_, total_access_size / min_task_access_size);
|
||||
tmp_total_task_count = min(params.min_task_count_per_thread_ * params.parallelism_,
|
||||
total_access_size/min_task_access_size);
|
||||
tmp_total_task_count = max(tmp_total_task_count, total_access_size / expected_task_load);
|
||||
LOG_TRACE("the data is less than lower bound size", K(ret), K(tmp_total_task_count));
|
||||
} else if (total_access_size > upper_bound_size) {
|
||||
@ -374,7 +345,8 @@ int ObGranuleUtil::compute_total_task_count(
|
||||
} else {
|
||||
// the data size is between lower bound size and upper bound size
|
||||
tmp_total_task_count = total_access_size / expected_task_load;
|
||||
LOG_TRACE("the data size is between lower bound size and upper bound size", K(ret), K(tmp_total_task_count));
|
||||
LOG_TRACE("the data size is between lower bound size and upper bound size",
|
||||
K(ret), K(tmp_total_task_count));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
@ -384,12 +356,14 @@ int ObGranuleUtil::compute_total_task_count(
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGranuleUtil::compute_task_count_each_partition(int64_t total_size, int64_t total_task_cnt,
|
||||
const common::ObIArray<int64_t>& size_each_partition, common::ObIArray<int64_t>& task_cnt_each_partition)
|
||||
int ObGranuleUtil::compute_task_count_each_partition(int64_t total_size,
|
||||
int64_t total_task_cnt,
|
||||
const common::ObIArray<int64_t> &size_each_partition,
|
||||
common::ObIArray<int64_t> &task_cnt_each_partition)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
// must ensure at least one task per partition.
|
||||
if (total_size <= 0 || total_task_cnt == size_each_partition.count()) {
|
||||
if (total_size <=0 || total_task_cnt == size_each_partition.count()) {
|
||||
// if the total count of tasks is equal to the number of partitions,
|
||||
// each partition just has one task.
|
||||
for (int i = 0; i < size_each_partition.count() && OB_SUCC(ret); i++) {
|
||||
@ -404,7 +378,7 @@ int ObGranuleUtil::compute_task_count_each_partition(int64_t total_size, int64_t
|
||||
int64_t alloc_task_cnt = 0;
|
||||
for (int i = 0; i < size_each_partition.count() && OB_SUCC(ret); i++) {
|
||||
int64_t partition_size = size_each_partition.at(i);
|
||||
int64_t task_cnt = ((double)partition_size / (double)total_size) * total_task_cnt;
|
||||
int64_t task_cnt = ((double) partition_size / (double) total_size) * total_task_cnt;
|
||||
// if the data volume of a partition is very small, but it still needs a task.
|
||||
if (task_cnt == 0) {
|
||||
task_cnt = 1;
|
||||
@ -414,15 +388,14 @@ int ObGranuleUtil::compute_task_count_each_partition(int64_t total_size, int64_t
|
||||
LOG_WARN("failed to push task cnt", K(ret));
|
||||
}
|
||||
}
|
||||
LOG_TRACE("compute task count for partition, allocate task count", K(ret), K(alloc_task_cnt), K(total_task_cnt));
|
||||
LOG_TRACE("compute task count for partition, allocate task count",
|
||||
K(ret), K(alloc_task_cnt), K(total_task_cnt));
|
||||
}
|
||||
// check the size of task_cnt_each_partition array
|
||||
if (OB_SUCC(ret) && task_cnt_each_partition.count() != size_each_partition.count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("the size of task count each partition is not right",
|
||||
K(ret),
|
||||
K(size_each_partition.count()),
|
||||
K(task_cnt_each_partition.count()));
|
||||
K(ret), K(size_each_partition.count()), K(task_cnt_each_partition.count()));
|
||||
}
|
||||
// check the returned result
|
||||
for (int i = 0; i < task_cnt_each_partition.count() && OB_SUCC(ret); i++) {
|
||||
@ -435,11 +408,18 @@ int ObGranuleUtil::compute_task_count_each_partition(int64_t total_size, int64_t
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObGranuleUtil::get_tasks_for_partition(ObIAllocator& allocator, int64_t expected_task_cnt, ObPartitionKey& pkey,
|
||||
storage::ObPartitionService& partition_service, ObIArray<ObStoreRange>& input_storage_ranges,
|
||||
ObIArray<ObNewRange>& tasks_ranges, ObIArray<int64_t>& tasks_offsets, ObIArray<int64_t>& tasks_partition_offsets)
|
||||
int ObGranuleUtil::get_tasks_for_partition(ObIAllocator &allocator,
|
||||
int64_t expected_task_cnt,
|
||||
ObDASTabletLoc &tablet,
|
||||
ObIArray<ObStoreRange> &input_storage_ranges,
|
||||
common::ObIArray<ObDASTabletLoc*> &granule_tablets,
|
||||
common::ObIArray<common::ObNewRange> &granule_ranges,
|
||||
common::ObIArray<int64_t> &granule_idx,
|
||||
int64_t &tablet_idx,
|
||||
bool range_independent)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObAccessService *access_service = MTL(ObAccessService *);
|
||||
ObArrayArray<ObStoreRange> multi_range_split_array;
|
||||
if (expected_task_cnt < 1) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
@ -449,56 +429,103 @@ int ObGranuleUtil::get_tasks_for_partition(ObIAllocator& allocator, int64_t expe
|
||||
for (int i = 0; i < input_storage_ranges.count() && OB_SUCC(ret); i++) {
|
||||
ObNewRange new_range;
|
||||
input_storage_ranges.at(i).to_new_range(new_range);
|
||||
if (OB_FAIL(tasks_ranges.push_back(new_range))) {
|
||||
if (OB_FAIL(granule_tablets.push_back(&tablet))) {
|
||||
LOG_WARN("failed to push back tablet", K(ret));
|
||||
} else if (OB_FAIL(granule_ranges.push_back(new_range))) {
|
||||
LOG_WARN("failed to push back range", K(ret));
|
||||
} else if (OB_FAIL(granule_idx.push_back(tablet_idx))) {
|
||||
LOG_WARN("failed to push back idx", K(ret));
|
||||
} else if (range_independent) {
|
||||
tablet_idx++;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(tasks_offsets.push_back(tasks_ranges.count() - 1))) {
|
||||
LOG_WARN("failed to push back offset", K(ret));
|
||||
} else if (OB_FAIL(tasks_partition_offsets.push_back(tasks_ranges.count() - 1))) {
|
||||
LOG_WARN("failed to push back partition offset", K(ret));
|
||||
}
|
||||
if (!range_independent) {
|
||||
tablet_idx++;
|
||||
}
|
||||
} else if (OB_FAIL(partition_service.split_multi_ranges(
|
||||
pkey, input_storage_ranges, expected_task_cnt, allocator, multi_range_split_array))) {
|
||||
LOG_WARN("failed to split multi ranges", K(ret), K(pkey), K(expected_task_cnt));
|
||||
} else if (OB_FAIL(access_service->split_multi_ranges(tablet.ls_id_,
|
||||
tablet.tablet_id_,
|
||||
input_storage_ranges,
|
||||
expected_task_cnt,
|
||||
allocator,
|
||||
multi_range_split_array))) {
|
||||
LOG_WARN("failed to split multi ranges", K(ret), K(tablet), K(expected_task_cnt));
|
||||
} else {
|
||||
LOG_TRACE("split multi ranges",
|
||||
K(ret),
|
||||
K(pkey),
|
||||
K(input_storage_ranges),
|
||||
K(expected_task_cnt == multi_range_split_array.count()),
|
||||
K(multi_range_split_array));
|
||||
K(ret), K(tablet), K(input_storage_ranges),
|
||||
K(expected_task_cnt == multi_range_split_array.count()), K(multi_range_split_array));
|
||||
// convert ObStoreRange array to ObNewRange array
|
||||
for (int i = 0; i < multi_range_split_array.count() && OB_SUCC(ret); i++) {
|
||||
ObIArray<ObStoreRange>& storage_task_ranges = multi_range_split_array.at(i);
|
||||
ObIArray<ObStoreRange> &storage_task_ranges = multi_range_split_array.at(i);
|
||||
for (int j = 0; j < storage_task_ranges.count() && OB_SUCC(ret); j++) {
|
||||
ObNewRange new_range;
|
||||
storage_task_ranges.at(j).to_new_range(new_range);
|
||||
if (OB_INVALID_INDEX == new_range.table_id_) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid table id", K(ret), K(new_range), K(multi_range_split_array.at(i)));
|
||||
} else if (OB_FAIL(tasks_ranges.push_back(new_range))) {
|
||||
} else if (OB_FAIL(granule_tablets.push_back(&tablet))) {
|
||||
LOG_WARN("failed to push back tablet", K(ret), K(tablet));
|
||||
} else if (OB_FAIL(granule_ranges.push_back(new_range))) {
|
||||
LOG_WARN("failed to push back new task range", K(ret), K(new_range));
|
||||
} else if (OB_FAIL(granule_idx.push_back(tablet_idx))) {
|
||||
LOG_WARN("failed to push back idx", K(ret), K(tablet_idx));
|
||||
} else if (range_independent) {
|
||||
tablet_idx++;
|
||||
}
|
||||
}
|
||||
// finish one task ranges, push the task offset
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(tasks_offsets.push_back(tasks_ranges.count() - 1))) {
|
||||
LOG_WARN("failed to push back tasks offset", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
// finish the whole partition ranges, push the partition task offset
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(tasks_partition_offsets.push_back(tasks_ranges.count() - 1))) {
|
||||
LOG_WARN("failed to push back tasks partition offset", K(ret));
|
||||
if (!range_independent) {
|
||||
tablet_idx++;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
int ObGranuleUtil::convert_new_range_to_store_range(ObIAllocator &allocator,
|
||||
const ObTableScanSpec *tsc,
|
||||
const ObTabletID &tablet_id,
|
||||
const ObIArray<ObNewRange> &input_ranges,
|
||||
ObIArray<ObStoreRange> &input_store_ranges,
|
||||
bool &need_convert_new_range)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObStoreRange store_range;
|
||||
input_store_ranges.reuse();
|
||||
need_convert_new_range = false;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < input_ranges.count(); i++) {
|
||||
if (input_ranges.at(i).is_physical_rowid_range_) {
|
||||
ObNewRange new_range;
|
||||
if (OB_ISNULL(tsc) || OB_UNLIKELY(tsc->get_columns_desc().count() < 1)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(ret), K(tsc));
|
||||
} else {
|
||||
ObArrayWrap<ObColDesc> rowkey_descs(&tsc->get_columns_desc().at(0),
|
||||
tsc->get_rowkey_cnt());
|
||||
if (OB_FAIL(deep_copy_range(allocator, input_ranges.at(i), new_range))) {
|
||||
LOG_WARN("failed to deep copy range", K(ret));
|
||||
} else if (OB_FAIL(ObTableScanOp::transform_physical_rowid(allocator,
|
||||
tablet_id,
|
||||
rowkey_descs,
|
||||
new_range))) {
|
||||
LOG_WARN("transform physical rowid for range failed", K(ret), K(new_range));
|
||||
} else {
|
||||
store_range.assign(new_range);
|
||||
if (OB_FAIL(input_store_ranges.push_back(store_range))) {
|
||||
LOG_WARN("failed to push back input store range", K(ret));
|
||||
} else {
|
||||
need_convert_new_range = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
store_range.assign(input_ranges.at(i));
|
||||
if (OB_FAIL(input_store_ranges.push_back(store_range))) {
|
||||
LOG_WARN("failed to push back input store range", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user