errsim to check the compare method for std::sort
This commit is contained in:
parent
5f52ab8cbb
commit
b7f4eb6aa2
@ -257,7 +257,7 @@ int ObLogDirScanner::search_log_dir_(const char *log_dir)
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(log_files.begin(), log_files.end());
|
||||
lib::ob_sort(log_files.begin(), log_files.end());
|
||||
ret = check_continuity_(log_files, min_log_id_, max_log_id_);
|
||||
}
|
||||
|
||||
|
2
deps/oblib/src/common/ob_common_utility.cpp
vendored
2
deps/oblib/src/common/ob_common_utility.cpp
vendored
@ -176,7 +176,7 @@ int64_t ObBasicTimeGuard::to_string(char *buf, const int64_t buf_len) const
|
||||
const int64_t click_count = click_count_ < MAX_CLICK_COUNT ? click_count_ : MAX_CLICK_COUNT;
|
||||
ClickInfo click_infos[click_count];
|
||||
MEMCPY(click_infos, click_infos_, sizeof(click_infos));
|
||||
std::sort(click_infos, click_infos + click_count, ClickInfo::compare);
|
||||
lib::ob_sort(click_infos, click_infos + click_count, ClickInfo::compare);
|
||||
ret = databuff_printf(buf, buf_len, pos,
|
||||
"owner: %s, click_count: %ld, time dist:[%s=%d",
|
||||
owner_, click_count_, click_infos[0].mod_, click_infos[0].cost_time_);
|
||||
|
2
deps/oblib/src/common/ob_learner_list.ipp
vendored
2
deps/oblib/src/common/ob_learner_list.ipp
vendored
@ -145,7 +145,7 @@ int BaseLearnerList<MAX_SIZE, T>::add_learner(const T &learner)
|
||||
} else if (OB_FAIL(learner_array_.push_back(learner))) {
|
||||
COMMON_LOG(ERROR, "learner_array_ push back failed", K(ret), K(learner));
|
||||
} else {
|
||||
std::sort(learner_array_.begin(), learner_array_.end(), [](const T &a, const T &b){ return a < b;});
|
||||
lib::ob_sort(learner_array_.begin(), learner_array_.end(), [](const T &a, const T &b){ return a < b;});
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
1
deps/oblib/src/common/ob_timeout_ctx.h
vendored
1
deps/oblib/src/common/ob_timeout_ctx.h
vendored
@ -15,6 +15,7 @@
|
||||
|
||||
#include "lib/ob_define.h"
|
||||
#include "lib/utility/ob_print_utils.h"
|
||||
#include "lib/time/ob_time_utility.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
|
@ -136,7 +136,7 @@ void ObRowChecksumValue::sort()
|
||||
}
|
||||
}
|
||||
if (!sorted) {
|
||||
std::sort(column_checksum_array_, column_checksum_array_ + column_count_);
|
||||
lib::ob_sort(column_checksum_array_, column_checksum_array_ + column_count_);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
6
deps/oblib/src/lib/alloc/memory_dump.cpp
vendored
6
deps/oblib/src/lib/alloc/memory_dump.cpp
vendored
@ -557,7 +557,7 @@ void ObMemoryDump::handle(void *task)
|
||||
} else if (STAT_LABEL == m_task->type_) {
|
||||
int tenant_cnt = 0;
|
||||
get_tenant_ids(tenant_ids_, MAX_TENANT_CNT, tenant_cnt);
|
||||
std::sort(tenant_ids_, tenant_ids_ + tenant_cnt);
|
||||
lib::ob_sort(tenant_ids_, tenant_ids_ + tenant_cnt);
|
||||
w_stat_->tcr_cnt_ = 0;
|
||||
w_stat_->malloc_sample_map_.clear();
|
||||
int64_t item_used = 0;
|
||||
@ -742,7 +742,7 @@ void ObMemoryDump::handle(void *task)
|
||||
if (m_task->dump_all_) {
|
||||
int tenant_cnt = 0;
|
||||
get_tenant_ids(tenant_ids_, MAX_TENANT_CNT, tenant_cnt);
|
||||
std::sort(tenant_ids_, tenant_ids_ + tenant_cnt);
|
||||
lib::ob_sort(tenant_ids_, tenant_ids_ + tenant_cnt);
|
||||
for (int tenant_idx = 0; tenant_idx < tenant_cnt; tenant_idx++) {
|
||||
uint64_t tenant_id = tenant_ids_[tenant_idx];
|
||||
for (int ctx_id = 0; ctx_id < ObCtxIds::MAX_CTX_ID; ctx_id++) {
|
||||
@ -775,7 +775,7 @@ void ObMemoryDump::handle(void *task)
|
||||
}
|
||||
LOG_INFO("chunk cnt", K(cnt));
|
||||
// sort chunk
|
||||
std::sort(chunks_, chunks_ + cnt);
|
||||
lib::ob_sort(chunks_, chunks_ + cnt);
|
||||
// iter chunk
|
||||
for (int i = 0; OB_SUCC(ret) && i < cnt; i++) {
|
||||
AChunk *chunk = chunks_[i];
|
||||
|
@ -103,7 +103,7 @@ int ObTenantCtxAllocator::iter_label(VisitFunc func) const
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(items, items + item_cnt,
|
||||
lib::ob_sort(items, items + item_cnt,
|
||||
[](ItemWrapper &l, ItemWrapper &r)
|
||||
{
|
||||
return (l.item_->hold_ > r.item_->hold_);
|
||||
|
@ -20,6 +20,7 @@
|
||||
#include "lib/time/ob_time_utility.h"
|
||||
#include "lib/resource/ob_resource_mgr.h"
|
||||
#include "lib/allocator/ob_tc_malloc.h"
|
||||
#include "lib/utility/ob_sort.h"
|
||||
#include <signal.h>
|
||||
|
||||
namespace oceanbase
|
||||
|
2
deps/oblib/src/lib/container/ob_vector.ipp
vendored
2
deps/oblib/src/lib/container/ob_vector.ipp
vendored
@ -587,7 +587,7 @@ template <typename T, typename Allocator>
|
||||
template <typename Compare>
|
||||
void ObSortedVector<T, Allocator>::sort(Compare compare)
|
||||
{
|
||||
std::sort(begin(), end(), compare);
|
||||
lib::ob_sort(begin(), end(), compare);
|
||||
}
|
||||
|
||||
} // end namespace common
|
||||
|
@ -374,7 +374,7 @@ int ObGeoInteriorPointVisitor::visit(ObIWkbGeomPolygon *geo)
|
||||
LOG_WARN("crossing_points size should be even", K(ret), K(crossing_points.size()));
|
||||
} else {
|
||||
double interior_x = 0;
|
||||
std::sort(crossing_points.begin(), crossing_points.end());
|
||||
lib::ob_sort(crossing_points.begin(), crossing_points.end());
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < crossing_points.size(); i += 2) {
|
||||
double width = crossing_points[i + 1] - crossing_points[i];
|
||||
if (width != 0 && width > max_width_) {
|
||||
|
@ -800,7 +800,7 @@ int ObJsonObject::rename_key(const common::ObString &old_key, const common::ObSt
|
||||
void ObJsonObject::sort()
|
||||
{
|
||||
ObJsonKeyCompare cmp;
|
||||
std::sort(object_array_.begin(), object_array_.end(), cmp);
|
||||
lib::ob_sort(object_array_.begin(), object_array_.end(), cmp);
|
||||
}
|
||||
|
||||
void ObJsonObject::stable_sort()
|
||||
|
2
deps/oblib/src/lib/lock/ob_bucket_lock.cpp
vendored
2
deps/oblib/src/lib/lock/ob_bucket_lock.cpp
vendored
@ -372,7 +372,7 @@ int ObMultiBucketLockGuard::lock_multi_buckets(ObIArray<uint64_t> &hash_array)
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(&latch_array_[0], &latch_array_[0] + latch_array_.count());
|
||||
lib::ob_sort(&latch_array_[0], &latch_array_[0] + latch_array_.count());
|
||||
for (; OB_SUCC(ret) && next_lock_idx < latch_array_.count(); ++next_lock_idx) {
|
||||
const uint64_t latch_idx = latch_array_.at(next_lock_idx);
|
||||
if (is_write_lock_) {
|
||||
|
2
deps/oblib/src/lib/restore/ob_storage.cpp
vendored
2
deps/oblib/src/lib/restore/ob_storage.cpp
vendored
@ -186,7 +186,7 @@ int ListAppendableObjectFragmentOp::gen_object_meta(ObStorageObjectMeta &obj_met
|
||||
int64_t last_end_offset = 0;
|
||||
obj_meta.reset();
|
||||
obj_meta.type_ = ObStorageObjectMetaType::OB_OBJ_SIMULATE_APPEND;
|
||||
std::sort(meta_arr_.begin(), meta_arr_.end(), ObStorageObjectMeta::fragment_meta_cmp_func);
|
||||
lib::ob_sort(meta_arr_.begin(), meta_arr_.end(), ObStorageObjectMeta::fragment_meta_cmp_func);
|
||||
while (OB_SUCC(ret) && i < meta_arr_.size()) {
|
||||
ObAppendableFragmentMeta &tmp_meta = meta_arr_[i];
|
||||
last_end_offset = tmp_meta.end_;
|
||||
|
37
deps/oblib/src/lib/utility/ob_print_utils.cpp
vendored
37
deps/oblib/src/lib/utility/ob_print_utils.cpp
vendored
@ -13,6 +13,7 @@
|
||||
#define USING_LOG_PREFIX LIB
|
||||
#include "lib/utility/ob_print_utils.h"
|
||||
#include "lib/string/ob_string.h"
|
||||
#include "lib/thread_local/ob_tsi_factory.h"
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace common
|
||||
@ -462,5 +463,41 @@ int databuff_print_key_obj(char *buf, const int64_t buf_len, int64_t &pos, const
|
||||
return ret;
|
||||
}
|
||||
|
||||
int64_t CStringBufMgr::acquire(char *&buffer)
|
||||
{
|
||||
int64_t buf_len = 0;
|
||||
if (0 == level_) {
|
||||
buffer = local_buf_ + pos_;
|
||||
buf_len = BUF_SIZE - pos_;
|
||||
} else {
|
||||
BufNode *node = NULL;
|
||||
node = list_.head_;
|
||||
if (NULL != node) {
|
||||
list_.head_ = node->pre_;
|
||||
}
|
||||
if ((NULL != node)
|
||||
|| (NULL != (node = OB_NEW(BufNode, ObModIds::OB_THREAD_BUFFER)))) {
|
||||
buffer = node->buf_ + node->pos_;
|
||||
node->pre_ = curr_;
|
||||
curr_ = node;
|
||||
buf_len = BUF_SIZE - node->pos_;
|
||||
} else {
|
||||
buffer = NULL;
|
||||
}
|
||||
}
|
||||
return buf_len;
|
||||
}
|
||||
|
||||
void CStringBufMgr::try_clear_list()
|
||||
{
|
||||
if (0 == level_) {
|
||||
while (NULL != list_.head_) {
|
||||
BufNode *node = list_.head_;
|
||||
list_.head_ = node->pre_;
|
||||
OB_DELETE(BufNode, ObModIds::OB_THREAD_BUFFER, node);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
} // end namespace common
|
||||
} // end namespace oceanbase
|
||||
|
40
deps/oblib/src/lib/utility/ob_print_utils.h
vendored
40
deps/oblib/src/lib/utility/ob_print_utils.h
vendored
@ -15,12 +15,12 @@
|
||||
#include "lib/ob_define.h"
|
||||
#include "lib/ob_name_def.h"
|
||||
#include "lib/string/ob_fixed_length_string.h"
|
||||
#include "lib/thread_local/ob_tsi_factory.h"
|
||||
#include "lib/hash_func/murmur_hash.h"
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace common
|
||||
{
|
||||
|
||||
class ObIAllocator;
|
||||
#define COMMA_FORMAT ", "
|
||||
#define WITH_COMMA(format) (with_comma ? COMMA_FORMAT format: format)
|
||||
|
||||
@ -189,40 +189,8 @@ public:
|
||||
curr_ = outer_layer;
|
||||
}
|
||||
}
|
||||
int64_t acquire(char *&buffer)
|
||||
{
|
||||
int64_t buf_len = 0;
|
||||
if (0 == level_) {
|
||||
buffer = local_buf_ + pos_;
|
||||
buf_len = BUF_SIZE - pos_;
|
||||
} else {
|
||||
BufNode *node = NULL;
|
||||
node = list_.head_;
|
||||
if (NULL != node) {
|
||||
list_.head_ = node->pre_;
|
||||
}
|
||||
if ((NULL != node)
|
||||
|| (NULL != (node = OB_NEW(BufNode, ObModIds::OB_THREAD_BUFFER)))) {
|
||||
buffer = node->buf_ + node->pos_;
|
||||
node->pre_ = curr_;
|
||||
curr_ = node;
|
||||
buf_len = BUF_SIZE - node->pos_;
|
||||
} else {
|
||||
buffer = NULL;
|
||||
}
|
||||
}
|
||||
return buf_len;
|
||||
}
|
||||
void try_clear_list()
|
||||
{
|
||||
if (0 == level_) {
|
||||
while (NULL != list_.head_) {
|
||||
BufNode *node = list_.head_;
|
||||
list_.head_ = node->pre_;
|
||||
OB_DELETE(BufNode, ObModIds::OB_THREAD_BUFFER, node);
|
||||
}
|
||||
}
|
||||
}
|
||||
int64_t acquire(char *&buffer);
|
||||
void try_clear_list();
|
||||
private:
|
||||
char local_buf_[BUF_SIZE];
|
||||
BufList list_;
|
||||
|
68
deps/oblib/src/lib/utility/ob_sort.h
vendored
Normal file
68
deps/oblib/src/lib/utility/ob_sort.h
vendored
Normal file
@ -0,0 +1,68 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SORT_WRAPPER_H_
|
||||
#define OCEANBASE_SORT_WRAPPER_H_
|
||||
|
||||
#include <algorithm>
|
||||
#include "lib/utility/ob_tracepoint.h"
|
||||
#include "lib/oblog/ob_log_module.h"
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace lib
|
||||
{
|
||||
|
||||
template <class T, class Compare>
|
||||
class CompareWrapper
|
||||
{
|
||||
public:
|
||||
CompareWrapper(Compare& comp)
|
||||
: comp_(comp)
|
||||
{}
|
||||
bool operator()(T &left, T &right)
|
||||
{
|
||||
int ret = OB_E(EventTable::EN_CHECK_SORT_CMP) OB_SUCCESS;
|
||||
if (OB_FAIL(ret)) {
|
||||
if (comp_(left, left) || comp_(right, right)) {
|
||||
OB_LOG_RET(ERROR, common::OB_ERR_UNEXPECTED,"check irreflexivity failed");
|
||||
}
|
||||
}
|
||||
return comp_(left, right);
|
||||
}
|
||||
private:
|
||||
Compare& comp_;
|
||||
};
|
||||
|
||||
template <class RandomAccessIterator, class Compare>
|
||||
void ob_sort(RandomAccessIterator first, RandomAccessIterator last, Compare comp)
|
||||
{
|
||||
using ValueType = typename std::iterator_traits<RandomAccessIterator>::value_type;
|
||||
CompareWrapper<ValueType, Compare> compare_wrapper(comp);
|
||||
std::sort(first, last, compare_wrapper);
|
||||
}
|
||||
|
||||
template <class RandomAccessIterator>
|
||||
void ob_sort(RandomAccessIterator first, RandomAccessIterator last)
|
||||
{
|
||||
using ValueType = typename std::iterator_traits<RandomAccessIterator>::value_type;
|
||||
struct Compare
|
||||
{
|
||||
bool operator()(ValueType& l, ValueType& r)
|
||||
{
|
||||
return l < r;
|
||||
}
|
||||
};
|
||||
ob_sort(first, last, Compare());
|
||||
}
|
||||
} // end of namespace lib
|
||||
} // end of namespace oceanbase
|
||||
#endif
|
@ -555,4 +555,6 @@ GLOBAL_ERRSIM_POINT_DEF(2451, COM_STMT_PREXECUTE_PS_CURSOR_OPEN_ERROR, "inject e
|
||||
GLOBAL_ERRSIM_POINT_DEF(2452, COM_STMT_PREXECUTE_EXECUTE_ERROR, "inject error at execute stage for pr-exec protocol");
|
||||
// Protocol end
|
||||
|
||||
GLOBAL_ERRSIM_POINT_DEF(2501, EN_CHECK_SORT_CMP, "Used to check the legality of the compare method for std::sort");
|
||||
|
||||
#endif /*GLOBAL_ERRSIM_POINT_DEF*/
|
||||
|
2
deps/oblib/src/lib/xml/ob_xpath.cpp
vendored
2
deps/oblib/src/lib/xml/ob_xpath.cpp
vendored
@ -1973,7 +1973,7 @@ int ObPathVarObject::add(const common::ObString &key, ObDatum *value, bool with_
|
||||
LOG_WARN("fail to push back", K(ret));
|
||||
} else { // sort again.
|
||||
ObPathKeyCompare cmp;
|
||||
std::sort(object_array_.begin(), object_array_.end(), cmp);
|
||||
lib::ob_sort(object_array_.begin(), object_array_.end(), cmp);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
|
1
deps/oblib/src/rpc/frame/ob_req_transport.h
vendored
1
deps/oblib/src/rpc/frame/ob_req_transport.h
vendored
@ -21,6 +21,7 @@
|
||||
#include "lib/statistic_event/ob_stat_event.h"
|
||||
#include "lib/stat/ob_diagnose_info.h"
|
||||
#include "rpc/obrpc/ob_rpc_packet.h"
|
||||
#include "lib/allocator/ob_malloc.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
|
@ -115,7 +115,7 @@ int ObFileListOp::func(const dirent *entry)
|
||||
int ObFileListOp::get_file_list(common::ObIArray<int64_t> &array)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
std::sort(array_.begin(), array_.end());
|
||||
lib::ob_sort(array_.begin(), array_.end());
|
||||
if (array_.empty()) {
|
||||
ret = OB_ENTRY_NOT_EXIST;
|
||||
} else if (OB_FAIL(array.assign(array_))){
|
||||
|
@ -664,7 +664,7 @@ int ObLSArchiveTask::ArchiveDest::pop_fetch_log(ObArchiveLogFetchTask *&task)
|
||||
} else {
|
||||
wait_send_task_array_[0] = wait_send_task_array_[wait_send_task_count_ - 1];
|
||||
wait_send_task_count_--;
|
||||
std::sort(wait_send_task_array_, wait_send_task_array_ + wait_send_task_count_, LogFetchTaskCompare());
|
||||
lib::ob_sort(wait_send_task_array_, wait_send_task_array_ + wait_send_task_count_, LogFetchTaskCompare());
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -744,7 +744,7 @@ int ObLSArchiveTask::ArchiveDest::push_fetch_log(ObArchiveLogFetchTask &task)
|
||||
} else {
|
||||
wait_send_task_array_[wait_send_task_count_] = &task;
|
||||
wait_send_task_count_++;
|
||||
std::sort(wait_send_task_array_, wait_send_task_array_ + wait_send_task_count_, LogFetchTaskCompare());
|
||||
lib::ob_sort(wait_send_task_array_, wait_send_task_array_ + wait_send_task_count_, LogFetchTaskCompare());
|
||||
}
|
||||
ARCHIVE_LOG(INFO, "print push_fetch_log", K(task), KPC(this));
|
||||
return ret;
|
||||
|
@ -37,7 +37,7 @@ int ObCDCMallocSampleInfo::init(const ObMallocSampleMap &sample_map)
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(samples_.begin(), samples_.end(), ObCDCMallocSamplePairCompartor());
|
||||
lib::ob_sort(samples_.begin(), samples_.end(), ObCDCMallocSamplePairCompartor());
|
||||
}
|
||||
|
||||
return ret;
|
||||
|
@ -1130,7 +1130,7 @@ int ObCDCPartTransResolver::check_redo_log_list_(
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(
|
||||
lib::ob_sort(
|
||||
sorted_redo_lsn_arr_in_trans_log.begin(),
|
||||
sorted_redo_lsn_arr_in_trans_log.end(),
|
||||
CDCLSNComparator());
|
||||
|
@ -408,7 +408,7 @@ int TransCtx::prepare_(
|
||||
if (OB_SUCC(ret)
|
||||
&& OB_UNLIKELY(NULL != host_ && host_->need_sort_participant())
|
||||
&& participant_count_ > 0) {
|
||||
std::sort(participants_, participants_ + participant_count_, TransPartInfoCompare());
|
||||
lib::ob_sort(participants_, participants_ + participant_count_, TransPartInfoCompare());
|
||||
}
|
||||
|
||||
_TCTX_DSTAT("[PREPARE] TENANT_ID=%lu TRANS_ID=%ld SERVED_PARTICIPANTS=%ld/%ld",
|
||||
|
@ -1558,7 +1558,7 @@ int sort_and_unique_lsn_arr(ObLogLSNArray &lsn_arr)
|
||||
palf::LSN prev_lsn;
|
||||
|
||||
// sort lsn_arr
|
||||
std::sort(lsn_arr.begin(), lsn_arr.end(), CDCLSNComparator());
|
||||
lib::ob_sort(lsn_arr.begin(), lsn_arr.end(), CDCLSNComparator());
|
||||
// get duplicate misslog lsn idx
|
||||
for(int64_t idx = 0; OB_SUCC(ret) && idx < lsn_arr.count(); idx++) {
|
||||
palf::LSN &cur_lsn = lsn_arr[idx];
|
||||
|
@ -603,7 +603,7 @@ int sort_and_unique_array(ARRAY &arr, Comparator &comparator)
|
||||
|
||||
if (arr.count() > 1) {
|
||||
// sort lsn_arr
|
||||
std::sort(arr.begin(), arr.end(), comparator);
|
||||
lib::ob_sort(arr.begin(), arr.end(), comparator);
|
||||
auto prev = arr.at(0);
|
||||
// get duplicate misslog lsn idx
|
||||
for(int64_t idx = 1; OB_SUCC(ret) && idx < arr.count(); idx++) {
|
||||
|
@ -145,12 +145,12 @@ int LSSvrList::get_server_array_for_locate_start_lsn(ObIArray<common::ObAddr> &s
|
||||
|
||||
void LSSvrList::sort_by_priority_for_locate_start_log_id_(SvrItemArray &svr_items) const
|
||||
{
|
||||
std::sort(svr_items.begin(), svr_items.end(), LocateStartLogIdCompare());
|
||||
lib::ob_sort(svr_items.begin(), svr_items.end(), LocateStartLogIdCompare());
|
||||
}
|
||||
|
||||
void LSSvrList::sort_by_priority()
|
||||
{
|
||||
std::sort(svr_items_.begin(), svr_items_.end(), SvrItemCompare());
|
||||
lib::ob_sort(svr_items_.begin(), svr_items_.end(), SvrItemCompare());
|
||||
}
|
||||
|
||||
int LSSvrList::filter_by_svr_blacklist(const ObLogSvrBlacklist &svr_blacklist,
|
||||
|
@ -2480,7 +2480,7 @@ int LogConfigMgr::sync_get_committed_end_lsn_(const LogConfigChangeArgs &args,
|
||||
K_(palf_id), K_(self), K(new_paxos_replica_num), K(paxos_resp_cnt),
|
||||
K(new_log_sync_replica_num), K(log_sync_resp_cnt), K(conn_timeout_us));
|
||||
} else {
|
||||
std::sort(lsn_array, lsn_array + log_sync_resp_cnt, LSNCompare());
|
||||
lib::ob_sort(lsn_array, lsn_array + log_sync_resp_cnt, LSNCompare());
|
||||
committed_end_lsn = lsn_array[new_log_sync_replica_num / 2];
|
||||
}
|
||||
PALF_LOG(INFO, "sync_get_committed_end_lsn_ finish", K(ret), K_(palf_id), K_(self), K(args),
|
||||
|
@ -2875,7 +2875,7 @@ int LogSlidingWindow::get_majority_lsn_(const ObMemberList &member_list,
|
||||
PALF_LOG(WARN, "match_lsn_map do not reach majority", K(ret), K_(palf_id), K_(self),
|
||||
K(member_list), K(replica_num), K(valid_member_cnt));
|
||||
} else if (OB_SUCC(ret)) {
|
||||
std::sort(lsn_array, lsn_array + valid_member_cnt, LSNCompare());
|
||||
lib::ob_sort(lsn_array, lsn_array + valid_member_cnt, LSNCompare());
|
||||
assert(replica_num / 2 < OB_MAX_MEMBER_NUMBER);
|
||||
result_lsn = lsn_array[replica_num / 2];
|
||||
if (!result_lsn.is_valid()) {
|
||||
|
@ -759,7 +759,7 @@ int ObLogRestoreHandler::submit_sorted_task(ObFetchLogTask &task)
|
||||
} else if (OB_FAIL(context_.submit_array_.push_back(&task))) {
|
||||
CLOG_LOG(WARN, "push back failed", K(ret), K(task));
|
||||
} else {
|
||||
std::sort(context_.submit_array_.begin(), context_.submit_array_.end(), FetchLogTaskCompare());
|
||||
lib::ob_sort(context_.submit_array_.begin(), context_.submit_array_.end(), FetchLogTaskCompare());
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -793,7 +793,7 @@ int ObLogRestoreHandler::get_next_sorted_task(ObFetchLogTask *&task)
|
||||
ObFetchLogTask *tmp_task = NULL;
|
||||
context_.submit_array_.pop_back(tmp_task);
|
||||
context_.submit_array_.at(0) = tmp_task;
|
||||
std::sort(context_.submit_array_.begin(), context_.submit_array_.end(), FetchLogTaskCompare());
|
||||
lib::ob_sort(context_.submit_array_.begin(), context_.submit_array_.end(), FetchLogTaskCompare());
|
||||
task = first;
|
||||
}
|
||||
return ret;
|
||||
|
@ -2651,7 +2651,7 @@ class WhenSortedByMatcher {
|
||||
LhsStlContainerReference lhs_stl_container = LhsView::ConstReference(lhs);
|
||||
::std::vector<LhsValue> sorted_container(lhs_stl_container.begin(),
|
||||
lhs_stl_container.end());
|
||||
::std::sort(
|
||||
::lib::ob_sort(
|
||||
sorted_container.begin(), sorted_container.end(), comparator_);
|
||||
|
||||
if (!listener->IsInterested()) {
|
||||
|
@ -13,6 +13,7 @@
|
||||
#ifndef OB_JIT_ALLOCATOR_H
|
||||
#define OB_JIT_ALLOCATOR_H
|
||||
|
||||
#include <sys/mman.h>
|
||||
#include "lib/oblog/ob_log.h"
|
||||
#include "lib/alloc/alloc_assist.h"
|
||||
#include "lib/utility/ob_print_utils.h"
|
||||
|
@ -196,7 +196,7 @@ int ObNetEndpointIngressManager::update_ingress_plan(ObNetEndpointKVArray &updat
|
||||
if (remain_bw_limit <= 0) {
|
||||
remain_bw_limit = 0;
|
||||
}
|
||||
std::sort(predicted_bws, predicted_bws + valid_count);
|
||||
lib::ob_sort(predicted_bws, predicted_bws + valid_count);
|
||||
int64_t average = remain_bw_limit / valid_count;
|
||||
bool is_done = false;
|
||||
for (int i = 0; i < valid_count && !is_done; i++) {
|
||||
|
@ -225,7 +225,7 @@ int ObServerSchemaUpdater::batch_process_tasks(
|
||||
LOG_WARN("fail to assign task", KR(ret), "task_cnt", batch_tasks.count());
|
||||
} else {
|
||||
DEBUG_SYNC(BEFORE_SET_NEW_SCHEMA_VERSION);
|
||||
std::sort(tasks.begin(), tasks.end(), ObServerSchemaTask::greator_than);
|
||||
lib::ob_sort(tasks.begin(), tasks.end(), ObServerSchemaTask::greator_than);
|
||||
ObServerSchemaTask::TYPE type = tasks.at(0).type_;
|
||||
if ((ObServerSchemaTask::REFRESH == type || ObServerSchemaTask::RELEASE == type)
|
||||
&& (1 != tasks.count())) {
|
||||
|
@ -124,9 +124,9 @@ int ObHTableExplicitColumnTracker::init(const table::ObHTableFilter &htable_filt
|
||||
ColumnCount *end = &columns_.at(columns_.count() - 1);
|
||||
++end;
|
||||
if (common::ObQueryFlag::Reverse == tracker_scan_order_) {
|
||||
std::sort(&columns_.at(0), end, ColumnCountReverseComparator());
|
||||
lib::ob_sort(&columns_.at(0), end, ColumnCountReverseComparator());
|
||||
} else {
|
||||
std::sort(&columns_.at(0), end, ColumnCountComparator());
|
||||
lib::ob_sort(&columns_.at(0), end, ColumnCountComparator());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -386,7 +386,7 @@ int ObTableQueryAndMutateP::sort_qualifier(common::ObIArray<ColumnIdx> &columns,
|
||||
// sort qualifiers
|
||||
ColumnIdx *end = &columns.at(columns.count()-1);
|
||||
++end;
|
||||
std::sort(&columns.at(0), end, ColumnIdxComparator());
|
||||
lib::ob_sort(&columns.at(0), end, ColumnIdxComparator());
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
// check duplicated qualifiers
|
||||
|
@ -747,7 +747,7 @@ int ObTableLoadCoordinator::pre_merge_peers()
|
||||
allocator))) {
|
||||
LOG_WARN("fail to get committed trans ids", KR(ret));
|
||||
} else {
|
||||
std::sort(arg.committed_trans_id_array_.begin(), arg.committed_trans_id_array_.end());
|
||||
lib::ob_sort(arg.committed_trans_id_array_.begin(), arg.committed_trans_id_array_.end());
|
||||
}
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < all_addr_array.count(); ++i) {
|
||||
|
@ -117,7 +117,7 @@ int ObTableLoadErrorRowHandler::handle_update_row(
|
||||
LOG_WARN("invalid args", KR(ret));
|
||||
} else {
|
||||
int64_t duplicate_row_count = rows.count() - 1;
|
||||
std::sort(rows.begin(), rows.end(),
|
||||
lib::ob_sort(rows.begin(), rows.end(),
|
||||
[](const ObDirectLoadExternalRow *lhs, const ObDirectLoadExternalRow *rhs) {
|
||||
return lhs->seq_no_ < rhs->seq_no_;
|
||||
});
|
||||
@ -163,7 +163,7 @@ int ObTableLoadErrorRowHandler::handle_update_row(
|
||||
LOG_WARN("invalid args", KR(ret));
|
||||
} else {
|
||||
int64_t duplicate_row_count = rows.count() - 1;
|
||||
std::sort(rows.begin(), rows.end(),
|
||||
lib::ob_sort(rows.begin(), rows.end(),
|
||||
[](const ObDirectLoadMultipleDatumRow *lhs, const ObDirectLoadMultipleDatumRow *rhs) {
|
||||
return lhs->seq_no_ < rhs->seq_no_;
|
||||
});
|
||||
|
@ -199,7 +199,7 @@ private:
|
||||
int ret = OB_SUCCESS;
|
||||
// sort heap table array
|
||||
ObTableLoadMultipleHeapTableCompactCompare compare;
|
||||
std::sort(curr_round->begin(), curr_round->end(), compare);
|
||||
lib::ob_sort(curr_round->begin(), curr_round->end(), compare);
|
||||
// compact top merge_count_per_round heap table
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < mem_ctx_->table_data_desc_.merge_count_per_round_;
|
||||
++i) {
|
||||
|
@ -235,7 +235,7 @@ public:
|
||||
sstable_array.set_tenant_id(MTL_ID());
|
||||
// sort sstable
|
||||
ObTableLoadParallelMergeSSTableCompare compare;
|
||||
std::sort(tablet_ctx_->sstables_.begin(), tablet_ctx_->sstables_.end(), compare);
|
||||
lib::ob_sort(tablet_ctx_->sstables_.begin(), tablet_ctx_->sstables_.end(), compare);
|
||||
// collect merged sstables
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < merge_sstable_count; ++i) {
|
||||
ObDirectLoadMultipleSSTable *sstable = tablet_ctx_->sstables_.at(i);
|
||||
|
@ -315,7 +315,7 @@ int ObTableLoadPartitionLocation::init_all_leader_info(ObIAllocator &allocator)
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(sort_array.begin(), sort_array.end(), [](const ObTableLoadPartitionLocation::LeaderInfoForSort &a,
|
||||
lib::ob_sort(sort_array.begin(), sort_array.end(), [](const ObTableLoadPartitionLocation::LeaderInfoForSort &a,
|
||||
ObTableLoadPartitionLocation::LeaderInfoForSort &b) {
|
||||
return a.addr_ < b.addr_;
|
||||
});
|
||||
|
@ -366,7 +366,7 @@ int ObTableLoadStore::pre_merge(
|
||||
LOG_WARN("unexpected committed trans count", KR(ret), K(committed_trans_id_array),
|
||||
K(store_committed_trans_id_array));
|
||||
} else {
|
||||
std::sort(store_committed_trans_id_array.begin(), store_committed_trans_id_array.end());
|
||||
lib::ob_sort(store_committed_trans_id_array.begin(), store_committed_trans_id_array.end());
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < committed_trans_id_array.count(); ++i) {
|
||||
if (OB_UNLIKELY(committed_trans_id_array[i] != store_committed_trans_id_array[i])) {
|
||||
ret = OB_ITEM_NOT_MATCH;
|
||||
|
@ -431,7 +431,7 @@ int ObGvSqlAudit::extract_tenant_ids()
|
||||
if (is_always_false) {
|
||||
tenant_id_array_.reset();
|
||||
} else {
|
||||
std::sort(tenant_id_array_.begin(), tenant_id_array_.end());
|
||||
lib::ob_sort(tenant_id_array_.begin(), tenant_id_array_.end());
|
||||
SERVER_LOG(DEBUG, "get tenant ids from req mgr map", K(tenant_id_array_));
|
||||
}
|
||||
}
|
||||
@ -499,7 +499,7 @@ int ObGvSqlAudit::extract_tenant_ids()
|
||||
if (is_always_false) {
|
||||
tenant_id_array_.reset();
|
||||
} else {
|
||||
std::sort(tenant_id_array_.begin(), tenant_id_array_.end());
|
||||
lib::ob_sort(tenant_id_array_.begin(), tenant_id_array_.end());
|
||||
SERVER_LOG(DEBUG, "get tenant ids from req mgr map", K(tenant_id_array_));
|
||||
}
|
||||
}
|
||||
|
@ -121,9 +121,9 @@ int ObIteratePrivateVirtualTable::do_open()
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (scan_flag_.is_reverse_scan()) {
|
||||
std::sort(tenants_.begin(), tenants_.end(), std::greater<uint64_t>());
|
||||
lib::ob_sort(tenants_.begin(), tenants_.end(), std::greater<uint64_t>());
|
||||
} else {
|
||||
std::sort(tenants_.begin(), tenants_.end());
|
||||
lib::ob_sort(tenants_.begin(), tenants_.end());
|
||||
}
|
||||
LOG_TRACE("tenant id array", K(tenants_));
|
||||
}
|
||||
|
@ -116,9 +116,9 @@ int ObIterateVirtualTable::do_open()
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (scan_flag_.is_reverse_scan()) {
|
||||
std::sort(tenants_.begin(), tenants_.end(), std::greater<uint64_t>());
|
||||
lib::ob_sort(tenants_.begin(), tenants_.end(), std::greater<uint64_t>());
|
||||
} else {
|
||||
std::sort(tenants_.begin(), tenants_.end());
|
||||
lib::ob_sort(tenants_.begin(), tenants_.end());
|
||||
}
|
||||
LOG_DEBUG("tenant id array", K(tenants_));
|
||||
}
|
||||
|
@ -265,7 +265,7 @@ int ObShowGrants::inner_get_next_row(common::ObNewRow *&row)
|
||||
LOG_WARN("push back failed", K(ret));
|
||||
}
|
||||
}
|
||||
std::sort(priv_key_array.begin(), priv_key_array.end(), PrivKey::cmp);
|
||||
lib::ob_sort(priv_key_array.begin(), priv_key_array.end(), PrivKey::cmp);
|
||||
ObArray<std::pair<PrivKey, ObPrivSet>> priv_key_value_set;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < priv_key_array.count(); i++) {
|
||||
bool need_print = false;
|
||||
|
@ -332,7 +332,7 @@ int ObVirtualShowTrace::merge_span_info() {
|
||||
// do nothing
|
||||
} else {
|
||||
ObSEArray<sql::ObFLTShowTraceRec*, 16> tmp_arr;
|
||||
std::sort(&show_trace_arr_.at(0), &show_trace_arr_.at(0) + show_trace_arr_.count(),
|
||||
lib::ob_sort(&show_trace_arr_.at(0), &show_trace_arr_.at(0) + show_trace_arr_.count(),
|
||||
[](const sql::ObFLTShowTraceRec* rec1, const sql::ObFLTShowTraceRec* rec2) {
|
||||
if (NULL == rec1) {
|
||||
return true;
|
||||
@ -570,7 +570,7 @@ int ObVirtualShowTrace::find_child_span_info(sql::ObFLTShowTraceRec::trace_forma
|
||||
if (tmp_arr.count() == 0) {
|
||||
// skipp sort
|
||||
} else {
|
||||
std::sort(&tmp_arr.at(0), &tmp_arr.at(0) + tmp_arr.count(),
|
||||
lib::ob_sort(&tmp_arr.at(0), &tmp_arr.at(0) + tmp_arr.count(),
|
||||
[](const sql::ObFLTShowTraceRec *rec1, const sql::ObFLTShowTraceRec *rec2) {
|
||||
if (NULL == rec1) {
|
||||
return true;
|
||||
|
@ -493,7 +493,7 @@ int ObVirtualSqlPlanMonitor::extract_tenant_ids()
|
||||
if (is_always_false) {
|
||||
tenant_id_array_.reset();
|
||||
} else {
|
||||
std::sort(tenant_id_array_.begin(), tenant_id_array_.end());
|
||||
lib::ob_sort(tenant_id_array_.begin(), tenant_id_array_.end());
|
||||
SERVER_LOG(DEBUG, "get tenant ids from req mgr map", K(tenant_id_array_));
|
||||
}
|
||||
}
|
||||
@ -563,7 +563,7 @@ int ObVirtualSqlPlanMonitor::extract_tenant_ids()
|
||||
if (is_always_false) {
|
||||
tenant_id_array_.reset();
|
||||
} else {
|
||||
std::sort(tenant_id_array_.begin(), tenant_id_array_.end());
|
||||
lib::ob_sort(tenant_id_array_.begin(), tenant_id_array_.end());
|
||||
SERVER_LOG(DEBUG, "get tenant ids from req mgr map", K(tenant_id_array_));
|
||||
}
|
||||
}
|
||||
|
@ -281,7 +281,7 @@ int ObDBMSLimitCalculator::get_max_value_of_logical_res_(
|
||||
}
|
||||
}//end for j for get each logical resource of servers
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(tmp_logical_resource.begin(), tmp_logical_resource.end());
|
||||
lib::ob_sort(tmp_logical_resource.begin(), tmp_logical_resource.end());
|
||||
int64_t index = 0;//下标
|
||||
while (OB_SUCC(ret) && index < server_cnt) {
|
||||
const int64_t logical_index = primary_server_cnt - 1 - index;
|
||||
|
@ -1165,7 +1165,7 @@ int ObDbmsWorkloadRepository::print_ash_top_exec_phase(const AshReportParams &as
|
||||
LOG_WARN("failed to push pair into phase array",K(ret));
|
||||
}
|
||||
|
||||
std::sort(phase_array.begin(), phase_array.end(), phase_cmp_func);
|
||||
lib::ob_sort(phase_array.begin(), phase_array.end(), phase_cmp_func);
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < phase_array.count(); i++) {
|
||||
const char *phase_name = phase_array.at(i).first;
|
||||
int64_t phase_cnt = phase_array.at(i).second;
|
||||
|
@ -1287,7 +1287,7 @@ int ObUserTenantBackupDeleteMgr::get_delete_obsolete_backup_set_infos_(
|
||||
} else if (OB_FAIL(ObBackupSetFileOperator::get_candidate_obsolete_backup_sets(*sql_proxy_, job_attr_->tenant_id_,
|
||||
job_attr_->expired_time_, backup_path_str.ptr(), backup_set_infos))) {
|
||||
LOG_WARN("failed to get candidate obsolete backup sets", K(ret));
|
||||
} else if (FALSE_IT(std::sort(backup_set_infos.begin(), backup_set_infos.end(), backup_set_info_cmp))) {
|
||||
} else if (FALSE_IT(lib::ob_sort(backup_set_infos.begin(), backup_set_infos.end(), backup_set_info_cmp))) {
|
||||
} else {
|
||||
for (int64_t i = backup_set_infos.count() - 1 ; OB_SUCC(ret) && i >= 0; i--) {
|
||||
bool need_deleted = false;
|
||||
@ -1380,7 +1380,7 @@ int ObUserTenantBackupDeleteMgr::get_delete_obsolete_backup_piece_infos_(const O
|
||||
ObArray<ObTenantArchivePieceAttr> backup_piece_infos;
|
||||
if (OB_FAIL(get_all_dest_backup_piece_infos_(clog_data_clean_point, backup_piece_infos))) {
|
||||
LOG_WARN("failed to get all dest backup piece infos", K(ret), K(clog_data_clean_point));
|
||||
} else if (FALSE_IT(std::sort(backup_piece_infos.begin(), backup_piece_infos.end(), backup_piece_info_cmp))) {
|
||||
} else if (FALSE_IT(lib::ob_sort(backup_piece_infos.begin(), backup_piece_infos.end(), backup_piece_info_cmp))) {
|
||||
} else {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < backup_piece_infos.count(); i++) {
|
||||
const ObTenantArchivePieceAttr &backup_piece_info = backup_piece_infos.at(i);
|
||||
|
@ -916,7 +916,7 @@ int ObBackupSetTaskMgr::fill_map_with_sys_tablets_(
|
||||
ObLSID sys_ls_id(ObLSID::SYS_LS_ID);
|
||||
if (OB_FAIL(get_extern_tablet_info_(sys_ls_id, tablet_ids, backup_scn))) {
|
||||
LOG_WARN("failed to get extern sys ls tablet info", K(ret));
|
||||
} else if (OB_FALSE_IT(std::sort(tablet_ids.begin(), tablet_ids.end()))) {
|
||||
} else if (OB_FALSE_IT(lib::ob_sort(tablet_ids.begin(), tablet_ids.end()))) {
|
||||
} else if (OB_FAIL(latest_ls_tablet_map.set_refactored(sys_ls_id, tablet_ids, 1))) {
|
||||
LOG_WARN("failed to set refactored", K(ret), K(sys_ls_id), K(tablet_ids));
|
||||
} else {
|
||||
@ -1581,7 +1581,7 @@ int ObBackupSetTaskMgr::do_get_change_turn_tablets_(
|
||||
LOG_WARN("[DATA_BACKUP]failed to get tablet to ls", K(ret), K(ls_info.ls_id_));
|
||||
} else if (OB_FAIL(append(ls_info.tablet_id_list_, map_iter->second))) {
|
||||
LOG_WARN("[DATA_BACKUP]failed to append tablet to ls array", K(ret));
|
||||
} else if (OB_FALSE_IT(std::sort(ls_info.tablet_id_list_.begin(), ls_info.tablet_id_list_.end()))) {
|
||||
} else if (OB_FALSE_IT(lib::ob_sort(ls_info.tablet_id_list_.begin(), ls_info.tablet_id_list_.end()))) {
|
||||
} else if (OB_FAIL(tablet_to_ls.push_back(ls_info))) {
|
||||
LOG_WARN("[DATA_BACKUP]failed to push backup ls info", K(ret));
|
||||
}
|
||||
|
@ -182,7 +182,7 @@ static int record_piece_info(const ObDestRoundCheckpointer::GeneratedPiece &piec
|
||||
single_ls_desc.round_id_, single_ls_desc.piece_id_, single_ls_desc.ls_id_,
|
||||
single_ls_desc.filelist_))) {
|
||||
LOG_WARN("failed to get archive file list", K(ret), K(single_ls_desc));
|
||||
} else if (OB_FALSE_IT(std::sort(single_ls_desc.filelist_.begin(), single_ls_desc.filelist_.end()))) {
|
||||
} else if (OB_FALSE_IT(lib::ob_sort(single_ls_desc.filelist_.begin(), single_ls_desc.filelist_.end()))) {
|
||||
} else if (OB_FAIL(piece_info_desc.filelist_.push_back(single_ls_desc))) {
|
||||
LOG_WARN("failed to push backup single_ls_desc", K(ret), K(single_ls_desc), K(piece_info_desc));
|
||||
} else if (OB_FAIL(store.is_single_ls_info_file_exist(single_ls_desc.dest_id_, single_ls_desc.round_id_, single_ls_desc.piece_id_, single_ls_desc.ls_id_, is_exist))) {
|
||||
|
@ -1930,7 +1930,7 @@ int check_trans_end(const ObArray<SendItem> &send_array,
|
||||
LOG_WARN("create return code map failed", K(ret));
|
||||
} else {
|
||||
// group by leader addr and send batch rpc
|
||||
std::sort(tmp_send_array.begin(), tmp_send_array.end());
|
||||
lib::ob_sort(tmp_send_array.begin(), tmp_send_array.end());
|
||||
ObAddr last_addr;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < tmp_send_array.count(); ++i) {
|
||||
const SendItem &send_item = tmp_send_array.at(i);
|
||||
@ -2629,7 +2629,7 @@ int ObDDLWaitColumnChecksumCtx::send_calc_rpc(int64_t &send_succ_count)
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
// group by leader addr and send batch rpc
|
||||
std::sort(send_array.begin(), send_array.end());
|
||||
lib::ob_sort(send_array.begin(), send_array.end());
|
||||
|
||||
ObAddr last_addr;
|
||||
int64_t group_start_idx = 0;
|
||||
|
@ -877,7 +877,7 @@ int ObNewTableTabletAllocator::alloc_tablet_for_add_balance_group(
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("final ls stat array count unexpected", KR(ret), K(final_ls_stat_array));
|
||||
} else {
|
||||
std::sort(final_ls_stat_array.begin(), final_ls_stat_array.end());
|
||||
lib::ob_sort(final_ls_stat_array.begin(), final_ls_stat_array.end());
|
||||
for (int64_t alloc_seq = 0; OB_SUCC(ret) && alloc_seq < total_alloc_num; alloc_seq++) {
|
||||
int64_t min_ls_tg_idx = 0;
|
||||
int64_t min_ls_tg_cnt = final_ls_stat_array.at(0).get_tablet_group_count();
|
||||
|
@ -79,7 +79,7 @@ ObBaseBootstrap::ObBaseBootstrap(ObSrvRpcProxy &rpc_proxy,
|
||||
rs_list_(rs_list),
|
||||
config_(config)
|
||||
{
|
||||
std::sort(rs_list_.begin(), rs_list_.end());
|
||||
lib::ob_sort(rs_list_.begin(), rs_list_.end());
|
||||
}
|
||||
|
||||
int ObBaseBootstrap::gen_sys_unit_ids(const ObIArray<ObZone> &zones,
|
||||
@ -94,7 +94,7 @@ int ObBaseBootstrap::gen_sys_unit_ids(const ObIArray<ObZone> &zones,
|
||||
} else if (OB_FAIL(sorted_zones.assign(zones))) {
|
||||
LOG_WARN("assign failed", K(ret));
|
||||
} else {
|
||||
std::sort(sorted_zones.begin(), sorted_zones.end());
|
||||
lib::ob_sort(sorted_zones.begin(), sorted_zones.end());
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < zones.count(); ++i) {
|
||||
for (int64_t j = 0; OB_SUCC(ret) && j < sorted_zones.count(); ++j) {
|
||||
if (sorted_zones.at(j) == zones.at(i)) {
|
||||
@ -664,7 +664,7 @@ int ObBootstrap::sort_schema(const ObIArray<ObTableSchema> &table_schemas,
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
TableIdCompare compare;
|
||||
std::sort(ptr_table_schemas.begin(), ptr_table_schemas.end(), compare);
|
||||
lib::ob_sort(ptr_table_schemas.begin(), ptr_table_schemas.end(), compare);
|
||||
if (OB_FAIL(compare.get_ret())) {
|
||||
LOG_WARN("fail to sort schema", KR(ret));
|
||||
} else {
|
||||
|
@ -17756,7 +17756,7 @@ int ObDDLService::gen_hidden_index_schema_columns(const ObTableSchema &orig_inde
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(index_columns.begin(), index_columns.end(),
|
||||
lib::ob_sort(index_columns.begin(), index_columns.end(),
|
||||
[](const std::pair<int64_t, obrpc::ObColumnSortItem> &lhs, const std::pair<int64_t, obrpc::ObColumnSortItem> &rhs) -> bool {
|
||||
return lhs.first < rhs.first;
|
||||
});
|
||||
@ -27341,7 +27341,7 @@ int ObDDLService::generate_zone_list_by_locality(
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(tmp_zone_list.begin(), tmp_zone_list.end());
|
||||
lib::ob_sort(tmp_zone_list.begin(), tmp_zone_list.end());
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < tmp_zone_list.count(); ++i) {
|
||||
common::ObZone &this_zone = tmp_zone_list.at(i);
|
||||
if (0 == zone_list.count() || zone_list.at(zone_list.count() - 1) != this_zone) {
|
||||
@ -27463,7 +27463,7 @@ int ObDDLService::get_zones_of_pools(
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("empty zone array", K(ret));
|
||||
} else {
|
||||
std::sort(temp_zones.begin(), temp_zones.end());
|
||||
lib::ob_sort(temp_zones.begin(), temp_zones.end());
|
||||
FOREACH_X(zone, temp_zones, OB_SUCC(ret)) {
|
||||
if (OB_ISNULL(zone)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
@ -27653,8 +27653,8 @@ int ObDDLService::modify_and_cal_resource_pool_diff(
|
||||
} else {} // no more to do
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(new_pool_name_list.begin(), new_pool_name_list.end());
|
||||
std::sort(old_pool_name_list.begin(), old_pool_name_list.end());
|
||||
lib::ob_sort(new_pool_name_list.begin(), new_pool_name_list.end());
|
||||
lib::ob_sort(old_pool_name_list.begin(), old_pool_name_list.end());
|
||||
bool is_permitted = false;
|
||||
if (new_pool_name_list.count() == old_pool_name_list.count() + 1) {
|
||||
grant = true;
|
||||
@ -30118,7 +30118,7 @@ int ObDDLService::check_create_tenant_locality(
|
||||
LOG_WARN("get_zones_of_pools return empty zone array", K(ret));
|
||||
} else {
|
||||
// get zones of resource pools, remove duplicated zone
|
||||
std::sort(temp_zones.begin(), temp_zones.end());
|
||||
lib::ob_sort(temp_zones.begin(), temp_zones.end());
|
||||
FOREACH_X(zone, temp_zones, OB_SUCC(ret)) {
|
||||
if (OB_ISNULL(zone)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
@ -35907,7 +35907,7 @@ int ObDDLService::check_schema_zone_list(
|
||||
common::ObArray<common::ObZone> &zone_list)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
std::sort(zone_list.begin(), zone_list.end());
|
||||
lib::ob_sort(zone_list.begin(), zone_list.end());
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < zone_list.count() - 1; ++i) {
|
||||
if (zone_list.at(i) == zone_list.at(i+1)) {
|
||||
ret = OB_ZONE_DUPLICATED;
|
||||
@ -36154,7 +36154,7 @@ int ObDDLService::trim_and_set_primary_zone(
|
||||
LOG_WARN("fail to check empty primary zone locality condition", K(ret));
|
||||
} else {} // no more to do
|
||||
} else {
|
||||
std::sort(new_zone_score_array.begin(), new_zone_score_array.end());
|
||||
lib::ob_sort(new_zone_score_array.begin(), new_zone_score_array.end());
|
||||
char primary_zone_str[MAX_ZONE_LENGTH];
|
||||
if (OB_FAIL(format_primary_zone_from_zone_score_array(
|
||||
new_zone_score_array, primary_zone_str, MAX_ZONE_LENGTH))) {
|
||||
|
@ -571,7 +571,7 @@ int ObDRWorker::LocalityAlignment::do_generate_locality_task_from_full_replica(
|
||||
}
|
||||
} else if (OB_SUCCESS == tmp_ret && nullptr != zone_replica_desc) {
|
||||
bool found = false;
|
||||
std::sort(zone_replica_desc->begin(), zone_replica_desc->end());
|
||||
lib::ob_sort(zone_replica_desc->begin(), zone_replica_desc->end());
|
||||
for (int64_t i = zone_replica_desc->count() - 1; !found && OB_SUCC(ret) && i >= 0; --i) {
|
||||
ReplicaDesc &replica_desc = zone_replica_desc->at(i);
|
||||
if (REPLICA_TYPE_FULL == replica_desc.replica_type_) {
|
||||
@ -643,7 +643,7 @@ int ObDRWorker::LocalityAlignment::do_generate_locality_task_from_logonly_replic
|
||||
}
|
||||
} else if (OB_SUCCESS == tmp_ret && nullptr != zone_replica_desc) {
|
||||
bool found = false;
|
||||
std::sort(zone_replica_desc->begin(), zone_replica_desc->end());
|
||||
lib::ob_sort(zone_replica_desc->begin(), zone_replica_desc->end());
|
||||
// defensive check
|
||||
for (int64_t i = zone_replica_desc->count() - 1; !found && OB_SUCC(ret) && i >= 0; --i) {
|
||||
ReplicaDesc &replica_desc = zone_replica_desc->at(i);
|
||||
@ -689,7 +689,7 @@ int ObDRWorker::LocalityAlignment::do_generate_locality_task_from_encryption_log
|
||||
}
|
||||
} else if (OB_SUCCESS == tmp_ret && nullptr != zone_replica_desc) {
|
||||
bool found = false;
|
||||
std::sort(zone_replica_desc->begin(), zone_replica_desc->end());
|
||||
lib::ob_sort(zone_replica_desc->begin(), zone_replica_desc->end());
|
||||
// defensive check
|
||||
for (int64_t i = zone_replica_desc->count() - 1; !found && OB_SUCC(ret) && i >= 0; --i) {
|
||||
ReplicaDesc &replica_desc = zone_replica_desc->at(i);
|
||||
@ -826,7 +826,7 @@ int ObDRWorker::LocalityAlignment::do_generate_locality_task_from_readonly_repli
|
||||
}
|
||||
} else if (OB_SUCCESS == tmp_ret && OB_NOT_NULL(zone_replica_desc)) {
|
||||
bool task_generated = false;
|
||||
std::sort(zone_replica_desc->begin(), zone_replica_desc->end());
|
||||
lib::ob_sort(zone_replica_desc->begin(), zone_replica_desc->end());
|
||||
// try to generate type_transform task if needed
|
||||
if (OB_FAIL(try_generate_type_transform_task_for_readonly_replica_(
|
||||
*zone_replica_desc, replica_stat_desc, index, task_generated))) {
|
||||
@ -1695,7 +1695,7 @@ bool ObDRWorker::LATaskCmp::operator()(const LATask *left, const LATask *right)
|
||||
|
||||
int ObDRWorker::LATaskCmp::execute_sort()
|
||||
{
|
||||
std::sort(task_array_.begin(), task_array_.end(), *this);
|
||||
lib::ob_sort(task_array_.begin(), task_array_.end(), *this);
|
||||
return ret_;
|
||||
}
|
||||
|
||||
|
@ -365,7 +365,7 @@ int ObLocalityDistribution::ZoneSetReplicaDist::try_set_specific_replica_dist(
|
||||
LOG_USER_ERROR(OB_INVALID_ARGUMENT,
|
||||
"locality, nonpaxos replica cannot specify multiple memstore percentage");
|
||||
} else {
|
||||
std::sort(replica_attr_set.begin(), replica_attr_set.end());
|
||||
lib::ob_sort(replica_attr_set.begin(), replica_attr_set.end());
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -380,7 +380,7 @@ int ObLocalityDistribution::ZoneSetReplicaDist::replace_zone_set(
|
||||
if (OB_FAIL(zone_set_.assign(this_zone_set))) {
|
||||
LOG_WARN("fail to assign zone set", K(ret));
|
||||
} else {
|
||||
std::sort(zone_set_.begin(), zone_set_.end());
|
||||
lib::ob_sort(zone_set_.begin(), zone_set_.end());
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -392,7 +392,7 @@ int ObLocalityDistribution::ZoneSetReplicaDist::append_zone(
|
||||
if (OB_FAIL(zone_set_.push_back(this_zone))) {
|
||||
LOG_WARN("fail to push back", K(ret));
|
||||
} else {
|
||||
std::sort(zone_set_.begin(), zone_set_.end());
|
||||
lib::ob_sort(zone_set_.begin(), zone_set_.end());
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -1505,7 +1505,7 @@ int ObLocalityDistribution::output_normalized_locality(
|
||||
// failed
|
||||
} else {
|
||||
bool start_format = false;
|
||||
std::sort(zone_set_replica_dist_array.begin(), zone_set_replica_dist_array.end());
|
||||
lib::ob_sort(zone_set_replica_dist_array.begin(), zone_set_replica_dist_array.end());
|
||||
for (int64_t i = 0; i < zone_set_replica_dist_array.count() && OB_SUCC(ret); ++i) {
|
||||
if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%s", (start_format ? ", " : "")))) {
|
||||
LOG_WARN("fail to format margin", K(ret));
|
||||
@ -1690,7 +1690,7 @@ int ObLocalityDistribution::get_zone_replica_attr_array(
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(sort_array.begin(), sort_array.end(), ObZoneReplicaAttrSet::sort_compare_less_than);
|
||||
lib::ob_sort(sort_array.begin(), sort_array.end(), ObZoneReplicaAttrSet::sort_compare_less_than);
|
||||
zone_replica_num_array.reset();
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < sort_array.count(); ++i) {
|
||||
if (OB_FAIL(zone_replica_num_array.push_back(*sort_array.at(i)))) {
|
||||
|
@ -834,7 +834,7 @@ int ObLSRecoveryStatHandler::calc_majority_min_readable_scn_(
|
||||
LOG_WARN("can not get majority readable_scn count", KR(ret),
|
||||
K(majority_cnt), K(readable_scn_list));
|
||||
} else {
|
||||
(void)std::sort(readable_scn_list.begin(), readable_scn_list.end(),
|
||||
(void)lib::ob_sort(readable_scn_list.begin(), readable_scn_list.end(),
|
||||
std::greater<share::SCN>());
|
||||
for (int64_t i = 0;
|
||||
OB_SUCC(ret) && i < readable_scn_list.count() && i < majority_cnt;
|
||||
|
@ -359,7 +359,7 @@ int ObMLogBuilder::MLogColumnUtils::construct_mlog_table_columns(
|
||||
return (a->get_column_id() < b->get_column_id());
|
||||
}
|
||||
};
|
||||
std::sort(mlog_table_column_array_.begin(), mlog_table_column_array_.end(), ColumnSchemaCmp());
|
||||
lib::ob_sort(mlog_table_column_array_.begin(), mlog_table_column_array_.end(), ColumnSchemaCmp());
|
||||
|
||||
for (int64_t i = 0; OB_SUCC(ret) && (i < mlog_table_column_array_.count()); ++i) {
|
||||
ObColumnSchemaV2 *column = mlog_table_column_array_.at(i);
|
||||
|
@ -339,7 +339,7 @@ int ObPartitionBalance::process_balance_partition_inner_()
|
||||
part_group_sum += ls_part_desc_arr.at(ls_idx)->get_part_groups().count();
|
||||
}
|
||||
while (OB_SUCC(ret)) {
|
||||
std::sort(ls_part_desc_arr.begin(), ls_part_desc_arr.end(), [] (ObLSPartGroupDesc* left, ObLSPartGroupDesc* right) {
|
||||
lib::ob_sort(ls_part_desc_arr.begin(), ls_part_desc_arr.end(), [] (ObLSPartGroupDesc* left, ObLSPartGroupDesc* right) {
|
||||
if (left->get_part_groups().count() < right->get_part_groups().count()) {
|
||||
return true;
|
||||
} else if (left->get_part_groups().count() == right->get_part_groups().count()) {
|
||||
@ -471,7 +471,7 @@ int ObPartitionBalance::process_balance_partition_extend_()
|
||||
part_group_sum += ls_desc_array_.at(i)->get_partgroup_cnt();
|
||||
}
|
||||
while (OB_SUCC(ret)) {
|
||||
std::sort(ls_desc_array_.begin(), ls_desc_array_.end(), [] (ObLSDesc *l, ObLSDesc *r) {
|
||||
lib::ob_sort(ls_desc_array_.begin(), ls_desc_array_.end(), [] (ObLSDesc *l, ObLSDesc *r) {
|
||||
if (l->get_partgroup_cnt() < r->get_partgroup_cnt()) {
|
||||
return true;
|
||||
} else if (l->get_partgroup_cnt() == r->get_partgroup_cnt()) {
|
||||
@ -571,7 +571,7 @@ int ObPartitionBalance::process_balance_partition_disk_()
|
||||
part_size_sum += ls_desc_array_.at(i)->get_data_size();
|
||||
}
|
||||
while (OB_SUCC(ret)) {
|
||||
std::sort(ls_desc_array_.begin(), ls_desc_array_.end(), [] (ObLSDesc *l, ObLSDesc *r) {
|
||||
lib::ob_sort(ls_desc_array_.begin(), ls_desc_array_.end(), [] (ObLSDesc *l, ObLSDesc *r) {
|
||||
if (l->get_data_size() < r->get_data_size()) {
|
||||
return true;
|
||||
} else if (l->get_data_size() == r->get_data_size()) {
|
||||
@ -649,10 +649,10 @@ int ObPartitionBalance::try_swap_part_group_(ObLSDesc &src_ls, ObLSDesc &dest_ls
|
||||
if (ls_more->get_part_groups().count() == 0 || ls_less->get_part_groups().count() == 0) {
|
||||
continue;
|
||||
}
|
||||
std::sort(ls_more->get_part_groups().begin(), ls_more->get_part_groups().end(), [] (ObTransferPartGroup *l, ObTransferPartGroup *r) {
|
||||
lib::ob_sort(ls_more->get_part_groups().begin(), ls_more->get_part_groups().end(), [] (ObTransferPartGroup *l, ObTransferPartGroup *r) {
|
||||
return l->get_data_size() < r->get_data_size();
|
||||
});
|
||||
std::sort(ls_less->get_part_groups().begin(), ls_less->get_part_groups().end(), [] (ObTransferPartGroup *l, ObTransferPartGroup *r) {
|
||||
lib::ob_sort(ls_less->get_part_groups().begin(), ls_less->get_part_groups().end(), [] (ObTransferPartGroup *l, ObTransferPartGroup *r) {
|
||||
return l->get_data_size() < r->get_data_size();
|
||||
});
|
||||
|
||||
|
@ -1106,7 +1106,7 @@ int ObLocalityCheckHelp::process_pre_single_zone_locality(
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
YIndexCmp cmp_operator;
|
||||
std::sort(pre_in_cur_multi_indexes.begin(), pre_in_cur_multi_indexes.end(), cmp_operator);
|
||||
lib::ob_sort(pre_in_cur_multi_indexes.begin(), pre_in_cur_multi_indexes.end(), cmp_operator);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -1156,7 +1156,7 @@ int ObLocalityCheckHelp::process_cur_single_zone_locality(
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
YIndexCmp cmp_operator;
|
||||
std::sort(cur_in_pre_multi_indexes.begin(), cur_in_pre_multi_indexes.end(), cmp_operator);
|
||||
lib::ob_sort(cur_in_pre_multi_indexes.begin(), cur_in_pre_multi_indexes.end(), cmp_operator);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
@ -1390,7 +1390,7 @@ int ObServerBalancer::try_balance_single_unit_by_disk(
|
||||
LOG_WARN("fail to push back", K(ret));
|
||||
} else {
|
||||
ServerTotalLoadCmp cmp;
|
||||
std::sort(server_load_ptrs_sorted.begin(), server_load_ptrs_sorted.end(), cmp);
|
||||
lib::ob_sort(server_load_ptrs_sorted.begin(), server_load_ptrs_sorted.end(), cmp);
|
||||
if (OB_FAIL(cmp.get_ret())) {
|
||||
LOG_WARN("fail to sort server loads", K(ret));
|
||||
}
|
||||
@ -1535,7 +1535,7 @@ int ObServerBalancer::try_balance_single_unit_by_cm(
|
||||
LOG_WARN("fail to push back", K(ret));
|
||||
} else {
|
||||
ServerTotalLoadCmp cmp;
|
||||
std::sort(server_load_ptrs_sorted.begin(), server_load_ptrs_sorted.end(), cmp);
|
||||
lib::ob_sort(server_load_ptrs_sorted.begin(), server_load_ptrs_sorted.end(), cmp);
|
||||
if (OB_FAIL(cmp.get_ret())) {
|
||||
LOG_WARN("fail to sort server loads", K(ret));
|
||||
}
|
||||
@ -1608,7 +1608,7 @@ int ObServerBalancer::do_balance_sys_tenant_units(
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
ServerTotalLoadCmp cmp;
|
||||
std::sort(server_load_ptrs_sorted.begin(), server_load_ptrs_sorted.end(), cmp);
|
||||
lib::ob_sort(server_load_ptrs_sorted.begin(), server_load_ptrs_sorted.end(), cmp);
|
||||
if (OB_FAIL(cmp.get_ret())) {
|
||||
LOG_WARN("fail to sort", K(ret));
|
||||
}
|
||||
@ -2902,7 +2902,7 @@ int ObServerBalancer::do_amend_inter_ttg_balance(
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
InterServerLoadCmp cmp;
|
||||
std::sort(server_load_ptrs_sorted.begin(), server_load_ptrs_sorted.end(), cmp);
|
||||
lib::ob_sort(server_load_ptrs_sorted.begin(), server_load_ptrs_sorted.end(), cmp);
|
||||
if (OB_FAIL(cmp.get_ret())) {
|
||||
LOG_WARN("fail to sort server load ptrs", K(ret));
|
||||
} else if (OB_FAIL(sort_server_loads_for_balance(
|
||||
@ -3192,7 +3192,7 @@ int ObServerBalancer::CountBalanceStrategy::try_exchange_ug_balance_inter_ttg_lo
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
UnitGroupLoadCmp unitgroup_load_cmp(left_inter_load);
|
||||
std::sort(left_inter_load.unitgroup_loads_.begin(),
|
||||
lib::ob_sort(left_inter_load.unitgroup_loads_.begin(),
|
||||
left_inter_load.unitgroup_loads_.end(),
|
||||
unitgroup_load_cmp);
|
||||
if (OB_FAIL(unitgroup_load_cmp.get_ret())) {
|
||||
@ -3201,7 +3201,7 @@ int ObServerBalancer::CountBalanceStrategy::try_exchange_ug_balance_inter_ttg_lo
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
UnitGroupLoadCmp unitgroup_load_cmp(right_inter_load);
|
||||
std::sort(right_inter_load.unitgroup_loads_.begin(),
|
||||
lib::ob_sort(right_inter_load.unitgroup_loads_.begin(),
|
||||
right_inter_load.unitgroup_loads_.end(),
|
||||
unitgroup_load_cmp);
|
||||
if (OB_FAIL(unitgroup_load_cmp.get_ret())) {
|
||||
@ -3265,7 +3265,7 @@ int ObServerBalancer::CountBalanceStrategy::try_move_single_ug_balance_inter_ttg
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
UnitGroupLoadCmp unitgroup_load_cmp(src_inter_load);
|
||||
std::sort(src_inter_load.unitgroup_loads_.begin(),
|
||||
lib::ob_sort(src_inter_load.unitgroup_loads_.begin(),
|
||||
src_inter_load.unitgroup_loads_.end(),
|
||||
unitgroup_load_cmp);
|
||||
if (OB_FAIL(unitgroup_load_cmp.get_ret())) {
|
||||
@ -3274,7 +3274,7 @@ int ObServerBalancer::CountBalanceStrategy::try_move_single_ug_balance_inter_ttg
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
UnitGroupLoadCmp unitgroup_load_cmp(dst_inter_load);
|
||||
std::sort(dst_inter_load.unitgroup_loads_.begin(),
|
||||
lib::ob_sort(dst_inter_load.unitgroup_loads_.begin(),
|
||||
dst_inter_load.unitgroup_loads_.end(),
|
||||
unitgroup_load_cmp);
|
||||
if (OB_FAIL(unitgroup_load_cmp.get_ret())) {
|
||||
@ -3995,7 +3995,7 @@ int ObServerBalancer::do_migrate_out_collide_unit(
|
||||
LOG_WARN("fail to push back", K(ret));
|
||||
} else {
|
||||
ServerTotalLoadCmp cmp;
|
||||
std::sort(server_load_ptrs_sorted.begin(), server_load_ptrs_sorted.end(), cmp);
|
||||
lib::ob_sort(server_load_ptrs_sorted.begin(), server_load_ptrs_sorted.end(), cmp);
|
||||
if (OB_FAIL(cmp.get_ret())) {
|
||||
LOG_WARN("fail to sort server loads", K(ret));
|
||||
}
|
||||
@ -4041,7 +4041,7 @@ int ObServerBalancer::pick_vacate_server_load(
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
ServerTotalLoadCmp cmp;
|
||||
if (FALSE_IT(std::sort(server_load_ptrs_sorted.begin(),
|
||||
if (FALSE_IT(lib::ob_sort(server_load_ptrs_sorted.begin(),
|
||||
server_load_ptrs_sorted.end(),
|
||||
cmp))) {
|
||||
} else if (OB_FAIL(cmp.get_ret())) {
|
||||
@ -4359,7 +4359,7 @@ int ObServerBalancer::vacate_space_by_single_unit(
|
||||
LOG_WARN("fail to push back", K(ret));
|
||||
} else {
|
||||
ServerTotalLoadCmp cmp;
|
||||
std::sort(server_load_ptrs_sorted.begin(), server_load_ptrs_sorted.end(), cmp);
|
||||
lib::ob_sort(server_load_ptrs_sorted.begin(), server_load_ptrs_sorted.end(), cmp);
|
||||
if (OB_FAIL(cmp.get_ret())) {
|
||||
LOG_WARN("fail to sort server loads", K(ret));
|
||||
}
|
||||
@ -4461,7 +4461,7 @@ int ObServerBalancer::generate_complete_server_loads(
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
ServerTotalLoadCmp cmp;
|
||||
std::sort(server_loads.begin(), server_loads.end(), cmp);
|
||||
lib::ob_sort(server_loads.begin(), server_loads.end(), cmp);
|
||||
if (OB_FAIL(cmp.get_ret())) {
|
||||
LOG_WARN("fail to sort", K(ret));
|
||||
}
|
||||
@ -4589,7 +4589,7 @@ int ObServerBalancer::make_single_wild_server_empty_by_units(
|
||||
} else if (OB_FAIL(get_pool_occupation_excluded_dst_servers(
|
||||
unit_load, pool_occupation, excluded_servers))) {
|
||||
LOG_WARN("fail to get excluded servers", K(ret));
|
||||
} else if (FALSE_IT(std::sort(available_server_loads.begin(),
|
||||
} else if (FALSE_IT(lib::ob_sort(available_server_loads.begin(),
|
||||
available_server_loads.end(),
|
||||
cmp))) {
|
||||
// sort cannot fail
|
||||
@ -4811,7 +4811,7 @@ int ObServerBalancer::make_available_servers_balance_by_disk(
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
UnitLoadDiskCmp cmp(unit_stat_mgr_);
|
||||
std::sort(standalone_unit_ptrs.begin(), standalone_unit_ptrs.end(), cmp);
|
||||
lib::ob_sort(standalone_unit_ptrs.begin(), standalone_unit_ptrs.end(), cmp);
|
||||
if (OB_FAIL(cmp.get_ret())) {
|
||||
LOG_WARN("fail to get ret", K(ret));
|
||||
}
|
||||
@ -4967,7 +4967,7 @@ int ObServerBalancer::generate_sort_available_servers_disk_statistic(
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
ServerDiskPercentCmp cmp;
|
||||
std::sort(available_servers_disk_statistic.begin(),
|
||||
lib::ob_sort(available_servers_disk_statistic.begin(),
|
||||
available_servers_disk_statistic.end(),
|
||||
cmp);
|
||||
if (OB_FAIL(cmp.get_ret())) {
|
||||
@ -5099,7 +5099,7 @@ int ObServerBalancer::make_server_disk_underload_by_unit(
|
||||
LOG_WARN("fail to update dst server load", K(ret));
|
||||
} else if (OB_FAIL(task_array.push_back(unit_migrate))) {
|
||||
LOG_WARN("fail to push back", K(ret));
|
||||
} else if (FALSE_IT(std::sort(under_disk_statistic.begin(),
|
||||
} else if (FALSE_IT(lib::ob_sort(under_disk_statistic.begin(),
|
||||
under_disk_statistic.end(),
|
||||
cmp))) {
|
||||
} else if (OB_FAIL(cmp.get_ret())) {
|
||||
@ -5136,7 +5136,7 @@ int ObServerBalancer::make_server_disk_underload(
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
UnitLoadDiskCmp cmp(unit_stat_mgr_);
|
||||
std::sort(standalone_unit_ptrs.begin(), standalone_unit_ptrs.end(), cmp);
|
||||
lib::ob_sort(standalone_unit_ptrs.begin(), standalone_unit_ptrs.end(), cmp);
|
||||
if (OB_FAIL(cmp.get_ret())) {
|
||||
LOG_WARN("fail to get ret", K(ret));
|
||||
}
|
||||
@ -6515,7 +6515,7 @@ int ObServerBalancer::CountBalanceStrategy::move_or_exchange_ug_balance_ttg_load
|
||||
double tolerance = sum_load / 100;
|
||||
// Sort by intra load descending order
|
||||
IntraServerLoadCmp cmp;
|
||||
std::sort(server_load_ptrs_sorted.begin(),
|
||||
lib::ob_sort(server_load_ptrs_sorted.begin(),
|
||||
server_load_ptrs_sorted.end(),
|
||||
cmp);
|
||||
if (OB_FAIL(cmp.get_ret())) {
|
||||
@ -6588,7 +6588,7 @@ int ObServerBalancer::CountBalanceStrategy::do_move_or_exchange_ug_balance_ttg_l
|
||||
} else {
|
||||
times++;
|
||||
IntraServerLoadCmp cmp;
|
||||
std::sort(server_load_ptrs_sorted.begin(),
|
||||
lib::ob_sort(server_load_ptrs_sorted.begin(),
|
||||
server_load_ptrs_sorted.end(),
|
||||
cmp);
|
||||
if (OB_FAIL(cmp.get_ret())) {
|
||||
@ -6637,7 +6637,7 @@ int ObServerBalancer::CountBalanceStrategy::try_move_ug_balance_ttg_load_foreach
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
UnitGroupLoadCmp unitgroup_load_cmp(max_server_load);
|
||||
std::sort(max_server_load.unitgroup_loads_.begin(),
|
||||
lib::ob_sort(max_server_load.unitgroup_loads_.begin(),
|
||||
max_server_load.unitgroup_loads_.end(),
|
||||
unitgroup_load_cmp);
|
||||
if (OB_FAIL(unitgroup_load_cmp.get_ret())) {
|
||||
@ -6646,7 +6646,7 @@ int ObServerBalancer::CountBalanceStrategy::try_move_ug_balance_ttg_load_foreach
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
UnitGroupLoadCmp unitgroup_load_cmp(min_server_load);
|
||||
std::sort(min_server_load.unitgroup_loads_.begin(),
|
||||
lib::ob_sort(min_server_load.unitgroup_loads_.begin(),
|
||||
min_server_load.unitgroup_loads_.end(),
|
||||
unitgroup_load_cmp);
|
||||
if (OB_FAIL(unitgroup_load_cmp.get_ret())) {
|
||||
@ -6727,7 +6727,7 @@ int ObServerBalancer::CountBalanceStrategy::try_exchange_ug_balance_ttg_load_for
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
UnitGroupLoadCmp unitgroup_load_cmp(max_server_load);
|
||||
std::sort(max_server_load.unitgroup_loads_.begin(),
|
||||
lib::ob_sort(max_server_load.unitgroup_loads_.begin(),
|
||||
max_server_load.unitgroup_loads_.end(),
|
||||
unitgroup_load_cmp);
|
||||
if (OB_FAIL(unitgroup_load_cmp.get_ret())) {
|
||||
@ -6736,7 +6736,7 @@ int ObServerBalancer::CountBalanceStrategy::try_exchange_ug_balance_ttg_load_for
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
UnitGroupLoadCmp unitgroup_load_cmp(min_server_load);
|
||||
std::sort(min_server_load.unitgroup_loads_.begin(),
|
||||
lib::ob_sort(min_server_load.unitgroup_loads_.begin(),
|
||||
min_server_load.unitgroup_loads_.end(),
|
||||
unitgroup_load_cmp);
|
||||
if (OB_FAIL(unitgroup_load_cmp.get_ret())) {
|
||||
@ -6771,7 +6771,7 @@ int ObServerBalancer::CountBalanceStrategy::make_count_balanced(
|
||||
if (OB_SUCC(ret)) {
|
||||
// Sort by the number of unitgroups on the server in ascending order
|
||||
ServerLoadUgCntCmp cmp;
|
||||
std::sort(server_cnt_ptrs_sorted.begin(),
|
||||
lib::ob_sort(server_cnt_ptrs_sorted.begin(),
|
||||
server_cnt_ptrs_sorted.end(),
|
||||
cmp);
|
||||
if (OB_FAIL(cmp.get_ret())) {
|
||||
@ -6814,7 +6814,7 @@ int ObServerBalancer::CountBalanceStrategy::do_make_count_balanced(
|
||||
} else {
|
||||
times++;
|
||||
ServerLoadUgCntCmp cmp;
|
||||
std::sort(server_cnt_ptrs_sorted.begin(),
|
||||
lib::ob_sort(server_cnt_ptrs_sorted.begin(),
|
||||
server_cnt_ptrs_sorted.end(),
|
||||
cmp);
|
||||
if (OB_FAIL(cmp.get_ret())) {
|
||||
@ -6859,7 +6859,7 @@ int ObServerBalancer::CountBalanceStrategy::do_make_count_balanced_foreach(
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
UnitGroupLoadCmp unitgroup_load_cmp(src_server_load);
|
||||
std::sort(src_server_load.unitgroup_loads_.begin(),
|
||||
lib::ob_sort(src_server_load.unitgroup_loads_.begin(),
|
||||
src_server_load.unitgroup_loads_.end(),
|
||||
unitgroup_load_cmp);
|
||||
if (OB_FAIL(unitgroup_load_cmp.get_ret())) {
|
||||
@ -6868,7 +6868,7 @@ int ObServerBalancer::CountBalanceStrategy::do_make_count_balanced_foreach(
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
UnitGroupLoadCmp unitgroup_load_cmp(dst_server_load);
|
||||
std::sort(dst_server_load.unitgroup_loads_.begin(),
|
||||
lib::ob_sort(dst_server_load.unitgroup_loads_.begin(),
|
||||
dst_server_load.unitgroup_loads_.end(),
|
||||
unitgroup_load_cmp);
|
||||
if (OB_FAIL(unitgroup_load_cmp.get_ret())) {
|
||||
@ -7945,7 +7945,7 @@ int ObServerBalancer::ZoneServerDiskStatistic::append(
|
||||
LOG_WARN("fail to push back", K(ret));
|
||||
} else {
|
||||
ServerDiskStatisticCmp cmp;
|
||||
std::sort(server_disk_statistic_array_.begin(),
|
||||
lib::ob_sort(server_disk_statistic_array_.begin(),
|
||||
server_disk_statistic_array_.end(),
|
||||
cmp);
|
||||
}
|
||||
@ -7979,7 +7979,7 @@ int ObServerBalancer::ZoneServerDiskStatistic::raise_server_disk_use(
|
||||
LOG_WARN("server not exist", K(ret), K(server));
|
||||
} else {
|
||||
ServerDiskStatisticCmp cmp;
|
||||
std::sort(server_disk_statistic_array_.begin(),
|
||||
lib::ob_sort(server_disk_statistic_array_.begin(),
|
||||
server_disk_statistic_array_.end(),
|
||||
cmp);
|
||||
}
|
||||
@ -8014,7 +8014,7 @@ int ObServerBalancer::ZoneServerDiskStatistic::reduce_server_disk_use(
|
||||
LOG_WARN("server not exist", K(ret), K(server));
|
||||
} else {
|
||||
ServerDiskStatisticCmp cmp;
|
||||
std::sort(server_disk_statistic_array_.begin(),
|
||||
lib::ob_sort(server_disk_statistic_array_.begin(),
|
||||
server_disk_statistic_array_.end(),
|
||||
cmp);
|
||||
}
|
||||
|
@ -227,7 +227,7 @@ int Matrix<T>::sort_column_group(
|
||||
} else {
|
||||
for (int64_t row = 0; OB_SUCC(ret) && row < row_count_; ++row) {
|
||||
const int64_t end_column_idx = start_column_idx + column_count;
|
||||
std::sort(inner_array_.begin() + row * column_count_ + start_column_idx,
|
||||
lib::ob_sort(inner_array_.begin() + row * column_count_ + start_column_idx,
|
||||
inner_array_.begin() + row * column_count_ + end_column_idx,
|
||||
func);
|
||||
if (OB_FAIL(func.get_ret())) {
|
||||
|
@ -550,7 +550,7 @@ int ObTenantTransferService::lock_table_and_part_(
|
||||
part_list.reset();
|
||||
ObSimpleTableSchemaV2 *table_schema = NULL;
|
||||
ObTransferPartInfo::Compare cmp;
|
||||
std::sort(ordered_part_list.begin(), ordered_part_list.end(), cmp);
|
||||
lib::ob_sort(ordered_part_list.begin(), ordered_part_list.end(), cmp);
|
||||
|
||||
ARRAY_FOREACH(ordered_part_list, idx) {
|
||||
ObLSID ls_id;
|
||||
@ -1419,7 +1419,7 @@ int ObTenantTransferService::unlock_table_and_part_(
|
||||
ObSimpleTableSchemaV2 *table_schema = NULL;
|
||||
const int64_t timeout_us = GCONF.internal_sql_execute_timeout;
|
||||
ObTransferPartInfo::Compare cmp;
|
||||
std::sort(ordered_part_list.begin(), ordered_part_list.end(), cmp);
|
||||
lib::ob_sort(ordered_part_list.begin(), ordered_part_list.end(), cmp);
|
||||
|
||||
ARRAY_FOREACH(ordered_part_list, idx) {
|
||||
ObTabletID tablet_id;
|
||||
|
@ -340,7 +340,7 @@ int ObTransferPartitionHelper::batch_get_table_schema_in_order_(
|
||||
} else {
|
||||
//按照table_id的顺序对table_schema_array进行排序
|
||||
ObBootstrap::TableIdCompare compare;
|
||||
std::sort(table_schema_array.begin(), table_schema_array.end(), compare);
|
||||
lib::ob_sort(table_schema_array.begin(), table_schema_array.end(), compare);
|
||||
if (OB_FAIL(compare.get_ret())) {
|
||||
LOG_WARN("failed to sort table schema", KR(ret), K(table_schema_array));
|
||||
}
|
||||
|
@ -73,7 +73,7 @@ int ObUnitManager::ZoneUnitPtr::assign(const ZoneUnitPtr &other)
|
||||
int ObUnitManager::ZoneUnitPtr::sort_by_unit_id_desc()
|
||||
{
|
||||
UnitGroupIdCmp cmp;
|
||||
std::sort(unit_ptrs_.begin(), unit_ptrs_.end(), cmp);
|
||||
lib::ob_sort(unit_ptrs_.begin(), unit_ptrs_.end(), cmp);
|
||||
return cmp.get_ret();
|
||||
}
|
||||
|
||||
@ -2276,7 +2276,7 @@ int ObUnitManager::get_to_be_deleted_unit_group(
|
||||
}
|
||||
}
|
||||
}
|
||||
std::sort(sorted_unit_group_id_array.begin(), sorted_unit_group_id_array.end());
|
||||
lib::ob_sort(sorted_unit_group_id_array.begin(), sorted_unit_group_id_array.end());
|
||||
// then other active unit group id
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < all_unit_group_id_array.count(); ++i) {
|
||||
uint64_t ug_id = all_unit_group_id_array.at(i);
|
||||
@ -6755,7 +6755,7 @@ int ObUnitManager::shrink_pool_unit_num(
|
||||
LOG_WARN("fail to push back", K(ret));
|
||||
}
|
||||
}
|
||||
std::sort(sort_unit_id_array.begin(), sort_unit_id_array.end());
|
||||
lib::ob_sort(sort_unit_id_array.begin(), sort_unit_id_array.end());
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < sort_unit_id_array.count() - 1; ++i) {
|
||||
if (sort_unit_id_array.at(i) == sort_unit_id_array.at(i + 1)) {
|
||||
ret = OB_NOT_SUPPORTED;
|
||||
@ -7815,7 +7815,7 @@ int ObUnitManager::alter_pool_zone_list(
|
||||
} else {}//no more to do
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(new_zone_list.begin(), new_zone_list.end());
|
||||
lib::ob_sort(new_zone_list.begin(), new_zone_list.end());
|
||||
bool is_add_pool_zone = false;
|
||||
bool is_remove_pool_zone = false;
|
||||
for (int64_t i = 0; i < new_zone_list.count() && OB_SUCC(ret); ++i) {
|
||||
@ -8718,8 +8718,8 @@ int ObUnitManager::construct_unit_group_id_for_unit_(
|
||||
LOG_WARN("fail to assign source unit group ids", KR(ret), K(source_unit_group_ids));
|
||||
} else {
|
||||
// construct sorted unit group ids first
|
||||
std::sort(sorted_new_ug_ids.begin(), sorted_new_ug_ids.end());
|
||||
std::sort(sorted_source_ug_ids.begin(), sorted_source_ug_ids.end());
|
||||
lib::ob_sort(sorted_new_ug_ids.begin(), sorted_new_ug_ids.end());
|
||||
lib::ob_sort(sorted_source_ug_ids.begin(), sorted_source_ug_ids.end());
|
||||
// for clone tenant, we have to construct unit_group_id for its units
|
||||
// the rule to allocate unit_group_id
|
||||
//
|
||||
@ -8973,7 +8973,7 @@ int ObUnitManager::build_zone_sorted_unit_array_(const share::ObResourcePool *po
|
||||
LOG_WARN("fail to assign zone unit array", KR(ret));
|
||||
} else {
|
||||
UnitZoneOrderCmp cmp_operator;
|
||||
std::sort(zone_sorted_units.begin(), zone_sorted_units.end(), cmp_operator);
|
||||
lib::ob_sort(zone_sorted_units.begin(), zone_sorted_units.end(), cmp_operator);
|
||||
// check unit count in each zone
|
||||
ObUnit *curr_unit = NULL;
|
||||
ObUnit *prev_unit = NULL;
|
||||
|
@ -153,7 +153,7 @@ int ObAliveZoneUnitsProvider::init(
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
UnitSortOp unit_sort_operator;
|
||||
std::sort(unit_ptr_array.begin(), unit_ptr_array.end(), unit_sort_operator);
|
||||
lib::ob_sort(unit_ptr_array.begin(), unit_ptr_array.end(), unit_sort_operator);
|
||||
if (OB_FAIL(unit_sort_operator.get_ret())) {
|
||||
LOG_WARN("fail to sort unit in zone", K(ret));
|
||||
} else if (OB_FAIL(all_zone_unit_ptrs_.push_back(unit_ptr_array))) {
|
||||
@ -163,7 +163,7 @@ int ObAliveZoneUnitsProvider::init(
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
ZoneUnitSortOp zone_unit_sort_operator;
|
||||
std::sort(all_zone_unit_ptrs_.begin(), all_zone_unit_ptrs_.end(), zone_unit_sort_operator);
|
||||
lib::ob_sort(all_zone_unit_ptrs_.begin(), all_zone_unit_ptrs_.end(), zone_unit_sort_operator);
|
||||
if (OB_FAIL(zone_unit_sort_operator.get_ret())) {
|
||||
LOG_WARN("fail to sort zone unit", K(ret));
|
||||
}
|
||||
|
@ -424,7 +424,7 @@ int ObDDLHelper::lock_objects_in_map_(
|
||||
}
|
||||
} // end foreach
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(lock_pairs.begin(), lock_pairs.end(), ObLockObjPair::less_than);
|
||||
lib::ob_sort(lock_pairs.begin(), lock_pairs.end(), ObLockObjPair::less_than);
|
||||
FOREACH_X(it, lock_pairs, OB_SUCC(ret)) {
|
||||
const int64_t timeout = ctx.get_timeout();
|
||||
if (OB_UNLIKELY(timeout <= 0)) {
|
||||
|
@ -774,7 +774,7 @@ int ObRestoreUtil::sort_backup_piece_array_(ObArray<share::ObSinglePieceDesc> &b
|
||||
LOG_WARN("empty backup_piece_array", K(ret));
|
||||
} else {
|
||||
ObSinglePieceDescComparator cmp;
|
||||
std::sort(backup_piece_array.begin(), backup_piece_array.end(), cmp);
|
||||
lib::ob_sort(backup_piece_array.begin(), backup_piece_array.end(), cmp);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -953,7 +953,7 @@ int ObRestoreUtil::sort_multi_paths_by_backup_set_id_(
|
||||
} else {
|
||||
multi_path_array.reset();
|
||||
|
||||
std::sort(
|
||||
lib::ob_sort(
|
||||
path_set_pairs.get_data(), path_set_pairs.get_data() + path_set_pairs.count(),
|
||||
[](const std::pair<ObString, ObBackupSetFileDesc> &a,
|
||||
const std::pair<ObString, ObBackupSetFileDesc> &b) { return a.second.backup_set_id_ < b.second.backup_set_id_; });
|
||||
@ -1278,7 +1278,7 @@ int ObRestoreUtil::get_all_piece_keys_(const ObIArray<ObString> &multi_path_arra
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
if (!piece_keys.empty()) {
|
||||
std::sort(piece_keys.begin(), piece_keys.end());
|
||||
lib::ob_sort(piece_keys.begin(), piece_keys.end());
|
||||
} else {
|
||||
ret = OB_RESTORE_SOURCE_NOT_ENOUGH;
|
||||
LOG_USER_ERROR(OB_RESTORE_SOURCE_NOT_ENOUGH, "no usable log pieces");
|
||||
|
@ -538,7 +538,7 @@ int ObArchiveStore::get_round_id(const int64_t dest_id, const SCN &scn, int64_t
|
||||
LOG_WARN("list files failed", K(ret), K(round_prefix), K(backup_dest));
|
||||
} else {
|
||||
ObArray<int64_t> &result = round_op.result();
|
||||
std::sort(result.begin(), result.end());
|
||||
lib::ob_sort(result.begin(), result.end());
|
||||
if (result.count() > 0) {
|
||||
round_id = result.at(0);
|
||||
} else {
|
||||
@ -589,7 +589,7 @@ int ObArchiveStore::get_all_round_ids(const int64_t dest_id, ObIArray<int64_t> &
|
||||
LOG_WARN("failed to list files", K(ret), K(round_prefix), K(backup_dest));
|
||||
} else {
|
||||
ObArray<int64_t> &result = round_op.result();
|
||||
std::sort(result.begin(), result.end());
|
||||
lib::ob_sort(result.begin(), result.end());
|
||||
if (OB_FAIL(roundid_array.assign(result))) {
|
||||
LOG_WARN("failed to assign round id array", K(ret));
|
||||
}
|
||||
@ -617,7 +617,7 @@ int ObArchiveStore::get_all_rounds(const int64_t dest_id, ObIArray<ObRoundEndDes
|
||||
LOG_WARN("list files failed", K(ret), K(round_prefix), K(backup_dest));
|
||||
} else {
|
||||
ObArray<ObRoundEndDesc> &result = round_op.result();
|
||||
std::sort(result.begin(), result.end());
|
||||
lib::ob_sort(result.begin(), result.end());
|
||||
if (OB_FAIL(rounds.assign(result))) {
|
||||
LOG_WARN("failed to assign rounds array", K(ret));
|
||||
}
|
||||
@ -756,7 +756,7 @@ int ObArchiveStore::get_piece_range(const int64_t dest_id, const int64_t round_i
|
||||
LOG_WARN("failed to list files", K(ret), K(piece_prefix), K(backup_dest));
|
||||
} else {
|
||||
ObArray<int64_t> &result = piece_op.result();
|
||||
std::sort(result.begin(), result.end());
|
||||
lib::ob_sort(result.begin(), result.end());
|
||||
if (result.count() > 0) {
|
||||
min_piece_id = result.at(0);
|
||||
max_piece_id = result.at(result.count() - 1);
|
||||
@ -1246,7 +1246,7 @@ int ObArchiveStore::get_all_piece_keys(ObIArray<ObPieceKey> &keys)
|
||||
LOG_WARN("failed to list files", K(ret), K(piece_prefix), K(backup_dest));
|
||||
} else {
|
||||
ObArray<ObPieceKey> &result = piece_op.result();
|
||||
std::sort(result.begin(), result.end());
|
||||
lib::ob_sort(result.begin(), result.end());
|
||||
if (OB_FAIL(keys.assign(result))) {
|
||||
LOG_WARN("failed to assign piece key array", K(ret));
|
||||
}
|
||||
@ -1573,7 +1573,7 @@ int ObArchiveStore::get_file_range_in_piece(const int64_t dest_id, const int64_t
|
||||
ret = OB_ENTRY_NOT_EXIST;
|
||||
LOG_WARN("file not exist", K(ret), K(dest_id), K(round_id), K(piece_id), K(ls_id));
|
||||
} else {
|
||||
std::sort(filelist.begin(), filelist.end());
|
||||
lib::ob_sort(filelist.begin(), filelist.end());
|
||||
min_file_id = filelist.at(0).file_id_;
|
||||
max_file_id = filelist.at(filelist.count() - 1).file_id_;
|
||||
}
|
||||
|
@ -119,7 +119,7 @@ int ObSortColumnIdArray::build_sort_array(
|
||||
}
|
||||
} // end of for
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(array_.begin(), array_.end());
|
||||
lib::ob_sort(array_.begin(), array_.end());
|
||||
LOG_TRACE("success to sort array", KR(ret), K(array_));
|
||||
get_func_ = get_func_from_array;
|
||||
}
|
||||
|
@ -92,7 +92,7 @@ void ObBaseConfig::get_sorted_config_items(ConfigItemArray &configs) const
|
||||
ConfigItem item(it->first.str(), NULL == it->second ? "" : it->second->str());
|
||||
(void)configs.push_back(item);
|
||||
}
|
||||
std::sort(configs.begin(), configs.end());
|
||||
lib::ob_sort(configs.begin(), configs.end());
|
||||
}
|
||||
int ObBaseConfig::load_from_buffer(const char *config_str, const int64_t config_str_len,
|
||||
const int64_t version, const bool check_name)
|
||||
|
@ -416,7 +416,7 @@ int ObExternalTableUtils::calc_assigned_files_to_sqcs(
|
||||
file_info.file_idx_ = i;
|
||||
OZ (sorted_files.push_back(file_info));
|
||||
}
|
||||
std::sort(sorted_files.begin(), sorted_files.end(),
|
||||
lib::ob_sort(sorted_files.begin(), sorted_files.end(),
|
||||
[](const FileInfoWithIdx &l, const FileInfoWithIdx &r) -> bool {
|
||||
return l.file_info_->file_size_ > r.file_info_->file_size_; });
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < sqc_count; i++) {
|
||||
@ -491,7 +491,7 @@ int ObExternalTableUtils::collect_local_files_on_servers(
|
||||
if (OB_SUCC(ret)) {
|
||||
if (is_absolute_path) {
|
||||
Functor functor;
|
||||
std::sort(all_servers.get_data(), all_servers.get_data() + all_servers.count(),
|
||||
lib::ob_sort(all_servers.get_data(), all_servers.get_data() + all_servers.count(),
|
||||
functor);
|
||||
ObAddr pre_addr;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < all_servers.count(); i++) {
|
||||
@ -603,7 +603,7 @@ int ObExternalTableUtils::sort_external_files(ObIArray<ObString> &file_urls,
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(file_urls.get_data(), file_urls.get_data() + file_urls.count());
|
||||
lib::ob_sort(file_urls.get_data(), file_urls.get_data() + file_urls.count());
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < file_urls.count(); ++i) {
|
||||
int64_t file_size = 0;
|
||||
if (OB_FAIL(file_map.get_refactored(file_urls.at(i), file_size))) {
|
||||
|
@ -172,7 +172,7 @@ int ObIOAbility::add_measure_item(const ObIOBenchResult &item)
|
||||
} else if (OB_FAIL(measure_items_[static_cast<int>(item.mode_)].push_back(item))) {
|
||||
LOG_WARN("push back measure_items failed", K(ret), K(item));
|
||||
} else {
|
||||
std::sort(measure_items_[static_cast<int>(item.mode_)].begin(), measure_items_[static_cast<int>(item.mode_)].end(),
|
||||
lib::ob_sort(measure_items_[static_cast<int>(item.mode_)].begin(), measure_items_[static_cast<int>(item.mode_)].end(),
|
||||
sort_fn);
|
||||
}
|
||||
return ret;
|
||||
|
@ -3665,7 +3665,7 @@ int64_t ObIOTracer::to_string(char *buf, const int64_t len) const
|
||||
} else if (OB_FAIL(counter.bt_count_.foreach_refactored(store_fn))) {
|
||||
LOG_WARN("get max backtrace count failed", K(ret));
|
||||
} else {
|
||||
std::sort(trace_array.begin(), trace_array.end(), sort_fn);
|
||||
lib::ob_sort(trace_array.begin(), trace_array.end(), sort_fn);
|
||||
databuff_printf(buf, len, pos, "trace_request_count: %ld, distinct_backtrace_count: %ld; ", counter.req_count_, trace_array.count());
|
||||
const int64_t print_count = min(5, trace_array.count());
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < print_count; ++i) {
|
||||
|
@ -83,7 +83,7 @@ int ObTabletLocationRefreshMgr::merge_inc_task_infos(
|
||||
if (inc_task_infos_to_merge.count() <= 0) {
|
||||
// do nothing
|
||||
} else {
|
||||
std::sort(inc_task_infos_to_merge.begin(), inc_task_infos_to_merge.end(), ObTransferRefreshInfo::less_than);
|
||||
lib::ob_sort(inc_task_infos_to_merge.begin(), inc_task_infos_to_merge.end(), ObTransferRefreshInfo::less_than);
|
||||
ObArray<ObTransferRefreshInfo> new_tasks;
|
||||
ObArray<ObTransferRefreshInfo> changed_tasks;
|
||||
|
||||
@ -134,7 +134,7 @@ int ObTabletLocationRefreshMgr::merge_inc_task_infos(
|
||||
} else if (OB_FAIL(append(inc_task_infos_, new_tasks))) {
|
||||
LOG_WARN("fail to append array", KR(ret), K(new_count));
|
||||
} else {
|
||||
std::sort(inc_task_infos_.begin(), inc_task_infos_.end(), ObTransferRefreshInfo::less_than);
|
||||
lib::ob_sort(inc_task_infos_.begin(), inc_task_infos_.end(), ObTransferRefreshInfo::less_than);
|
||||
FLOG_INFO("[REFRESH_TABLET_LOCATION] add tasks",
|
||||
KR(ret), K_(tenant_id), K(new_tasks.count()), K(new_tasks));
|
||||
}
|
||||
|
@ -55,7 +55,7 @@ int ObCoreTableProxy::Row::init(const int64_t row_id,
|
||||
cells_[cell_cnt_++] = *c;
|
||||
}
|
||||
if (cell_cnt_ > 0) {
|
||||
std::sort(cells_, cells_ + cell_cnt_);
|
||||
lib::ob_sort(cells_, cells_ + cell_cnt_);
|
||||
}
|
||||
inited_ = true;
|
||||
}
|
||||
@ -698,7 +698,7 @@ int ObCoreTableProxy::incremental_update(const ObIArray<UpdateCell> &cells,
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(new_cells.begin(), new_cells.end());
|
||||
lib::ob_sort(new_cells.begin(), new_cells.end());
|
||||
if (OB_FAIL(add_row(new_row_id, new_cells))) {
|
||||
LOG_WARN("add row failed", K(ret));
|
||||
}
|
||||
@ -764,7 +764,7 @@ int ObCoreTableProxy::update(const ObIArray<UpdateCell> &cells,
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(new_cells.begin(), new_cells.end());
|
||||
lib::ob_sort(new_cells.begin(), new_cells.end());
|
||||
if (OB_FAIL(add_row(new_row_id, new_cells))) {
|
||||
LOG_WARN("add row failed", K(ret));
|
||||
}
|
||||
|
@ -476,7 +476,7 @@ int ObDDLChecksumOperator::get_tablet_checksum_record_without_execution_id(
|
||||
if (OB_FAIL(batch_tablet_array.push_back(last_tablet_id_id))) {
|
||||
LOG_WARN("fail to push back tablet_id_id", K(ret), K(tenant_id), K(ddl_task_id), K(last_tablet_id_id));
|
||||
} else {
|
||||
std::sort(batch_tablet_array.begin(), batch_tablet_array.end());
|
||||
lib::ob_sort(batch_tablet_array.begin(), batch_tablet_array.end());
|
||||
if ((i != 0 && i % batch_size == 0) /* reach batch size */ || i == tablet_ids.count() - 1 /* reach end */) {
|
||||
if (OB_FAIL(sql.assign_fmt(
|
||||
"SELECT task_id FROM %s "
|
||||
|
@ -3051,7 +3051,7 @@ int ObCheckTabletDataComplementOp::check_finish_report_checksum(
|
||||
LOG_WARN("fail to check report checksum finished", K(ret), K(tenant_id), K(index_table_id), K(execution_id), K(ddl_task_id));
|
||||
} else if (OB_FAIL(ObDDLUtil::get_tablets(tenant_id, index_table_id, dest_tablet_ids))) {
|
||||
LOG_WARN("fail to get tablets", K(ret), K(tenant_id), K(index_table_id));
|
||||
} else if (OB_FALSE_IT(std::sort(dest_tablet_ids.begin(), dest_tablet_ids.end()))) { // sort in ASC order.
|
||||
} else if (OB_FALSE_IT(lib::ob_sort(dest_tablet_ids.begin(), dest_tablet_ids.end()))) { // sort in ASC order.
|
||||
} else if (OB_FAIL(check_tablet_checksum_update_status(tenant_id, index_table_id, ddl_task_id, execution_id, dest_tablet_ids, is_checksums_all_report))) {
|
||||
LOG_WARN("fail to check tablet checksum update status, maybe EAGAIN", K(ret), K(tenant_id), K(dest_tablet_ids), K(execution_id));
|
||||
} else if (!is_checksums_all_report) {
|
||||
|
@ -209,7 +209,7 @@ int ObFreezeInfoManager::update_freeze_info(
|
||||
} else if (OB_FAIL(freeze_info_.frozen_statuses_.assign(freeze_infos))) {
|
||||
LOG_WARN("fail to assign", KR(ret), K(freeze_infos));
|
||||
} else if (freeze_info_.frozen_statuses_.count() > 1) {
|
||||
std::sort(freeze_info_.frozen_statuses_.begin(), freeze_info_.frozen_statuses_.end(),
|
||||
lib::ob_sort(freeze_info_.frozen_statuses_.begin(), freeze_info_.frozen_statuses_.end(),
|
||||
[](const ObFreezeInfo &a, const ObFreezeInfo &b)
|
||||
{ return a.frozen_scn_ < b.frozen_scn_; } );
|
||||
}
|
||||
|
@ -179,7 +179,7 @@ int ObPrimaryZoneUtil::construct_zone_array()
|
||||
LOG_WARN("fail to push back", K(ret));
|
||||
} else {} // no more
|
||||
}
|
||||
std::sort(tmp_zone_array.begin(), tmp_zone_array.end());
|
||||
lib::ob_sort(tmp_zone_array.begin(), tmp_zone_array.end());
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < tmp_zone_array.count() - 1; ++i) {
|
||||
if (tmp_zone_array.at(i) == tmp_zone_array.at(i + 1)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
@ -445,7 +445,7 @@ int ObPrimaryZoneUtil::update_region_score()
|
||||
LOG_WARN("zone region list null", K(ret), KP(zone_region_list_));
|
||||
} else {
|
||||
RegionScoreCmp region_score_cmp;
|
||||
std::sort(normalized_zone_array_.begin(), normalized_zone_array_.end(), region_score_cmp);
|
||||
lib::ob_sort(normalized_zone_array_.begin(), normalized_zone_array_.end(), region_score_cmp);
|
||||
ObRegion last_region;
|
||||
int64_t last_region_score = INT64_MAX;
|
||||
for (int64_t i = 0; i < normalized_zone_array_.count() && OB_SUCC(ret); ++i) {
|
||||
@ -465,7 +465,7 @@ int ObPrimaryZoneUtil::construct_full_zone_array()
|
||||
int ret = OB_SUCCESS;
|
||||
// sorted by region_score,zone_score,zone, the zone priority is determined by now
|
||||
FinalCmp final_cmp;
|
||||
std::sort(normalized_zone_array_.begin(), normalized_zone_array_.end(), final_cmp);
|
||||
lib::ob_sort(normalized_zone_array_.begin(), normalized_zone_array_.end(), final_cmp);
|
||||
if (OB_UNLIKELY(normalized_zone_array_.count() <= 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("cannot construct full zone array", K(ret),
|
||||
|
@ -547,7 +547,7 @@ int ObZoneReplicaAttrSet::append(const ObZoneReplicaAttrSet &that)
|
||||
LOG_WARN("fail to push back", K(ret));
|
||||
}
|
||||
}
|
||||
std::sort(zone_set_.begin(), zone_set_.end());
|
||||
lib::ob_sort(zone_set_.begin(), zone_set_.end());
|
||||
for (int64_t i = 0;
|
||||
OB_SUCC(ret) && i < that.replica_attr_set_.get_full_replica_attr_array().count();
|
||||
++i) {
|
||||
@ -556,7 +556,7 @@ int ObZoneReplicaAttrSet::append(const ObZoneReplicaAttrSet &that)
|
||||
LOG_WARN("fail to add full replica num", K(ret));
|
||||
}
|
||||
}
|
||||
std::sort(replica_attr_set_.get_full_replica_attr_array_for_sort().begin(),
|
||||
lib::ob_sort(replica_attr_set_.get_full_replica_attr_array_for_sort().begin(),
|
||||
replica_attr_set_.get_full_replica_attr_array_for_sort().end());
|
||||
for (int64_t i = 0;
|
||||
OB_SUCC(ret) && i < that.replica_attr_set_.get_logonly_replica_attr_array().count();
|
||||
@ -566,7 +566,7 @@ int ObZoneReplicaAttrSet::append(const ObZoneReplicaAttrSet &that)
|
||||
LOG_WARN("fail to add logonly replica num", K(ret));
|
||||
}
|
||||
}
|
||||
std::sort(replica_attr_set_.get_logonly_replica_attr_array_for_sort().begin(),
|
||||
lib::ob_sort(replica_attr_set_.get_logonly_replica_attr_array_for_sort().begin(),
|
||||
replica_attr_set_.get_logonly_replica_attr_array_for_sort().end());
|
||||
for (int64_t i = 0;
|
||||
OB_SUCC(ret) && i < that.replica_attr_set_.get_readonly_replica_attr_array().count();
|
||||
@ -576,7 +576,7 @@ int ObZoneReplicaAttrSet::append(const ObZoneReplicaAttrSet &that)
|
||||
LOG_WARN("fail to add readonly replica num", K(ret));
|
||||
}
|
||||
}
|
||||
std::sort(replica_attr_set_.get_readonly_replica_attr_array_for_sort().begin(),
|
||||
lib::ob_sort(replica_attr_set_.get_readonly_replica_attr_array_for_sort().begin(),
|
||||
replica_attr_set_.get_readonly_replica_attr_array_for_sort().end());
|
||||
for (int64_t i = 0;
|
||||
OB_SUCC(ret) && i < that.replica_attr_set_.get_encryption_logonly_replica_attr_array().count();
|
||||
@ -587,7 +587,7 @@ int ObZoneReplicaAttrSet::append(const ObZoneReplicaAttrSet &that)
|
||||
LOG_WARN("fail to add logonly replica num", K(ret));
|
||||
}
|
||||
}
|
||||
std::sort(replica_attr_set_.get_encryption_logonly_replica_attr_array_for_sort().begin(),
|
||||
lib::ob_sort(replica_attr_set_.get_encryption_logonly_replica_attr_array_for_sort().begin(),
|
||||
replica_attr_set_.get_encryption_logonly_replica_attr_array_for_sort().end());
|
||||
return ret;
|
||||
}
|
||||
|
@ -418,7 +418,7 @@ int ObSchemaServiceSQLImpl::get_batch_table_schema(
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid schema_version", K(schema_version), K(ret));
|
||||
} else {
|
||||
std::sort(table_ids.begin(), table_ids.end(), std::greater<uint64_t>());
|
||||
lib::ob_sort(table_ids.begin(), table_ids.end(), std::greater<uint64_t>());
|
||||
// get not core table schemas from __all_table and __all_column
|
||||
if (OB_FAIL(get_not_core_table_schemas(schema_status, schema_version, table_ids,
|
||||
sql_client, allocator, table_schema_array))) {
|
||||
@ -662,7 +662,7 @@ int ObSchemaServiceSQLImpl::get_core_table_priorities(
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(temp_table_schema_ptrs.begin(), temp_table_schema_ptrs.end(), cmp_table_id);
|
||||
lib::ob_sort(temp_table_schema_ptrs.begin(), temp_table_schema_ptrs.end(), cmp_table_id);
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < temp_table_schema_ptrs.count(); ++i) {
|
||||
if (OB_FAIL(core_schemas.push_back(*(temp_table_schema_ptrs.at(i))))) {
|
||||
@ -3353,7 +3353,7 @@ int ObSchemaServiceSQLImpl::get_batch_tenants(
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("check inner stat fail");
|
||||
} else {
|
||||
std::sort(schema_keys.begin(), schema_keys.end(), SchemaKey::cmp_with_tenant_id);
|
||||
lib::ob_sort(schema_keys.begin(), schema_keys.end(), SchemaKey::cmp_with_tenant_id);
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -3393,7 +3393,7 @@ int ObSchemaServiceSQLImpl::get_batch_tenants(
|
||||
} else if (OB_FAIL(schema_array.reserve(schema_keys.count()))) {\
|
||||
LOG_WARN("fail to reserve schema array", KR(ret)); \
|
||||
} else { \
|
||||
std::sort(schema_keys.begin(), schema_keys.end(), SchemaKey::cmp_with_tenant_id); \
|
||||
lib::ob_sort(schema_keys.begin(), schema_keys.end(), SchemaKey::cmp_with_tenant_id); \
|
||||
int64_t begin = 0; \
|
||||
int64_t end = 0; \
|
||||
while (OB_SUCCESS == ret && end < schema_keys.count()) { \
|
||||
@ -3446,7 +3446,7 @@ GET_BATCH_SCHEMAS_WITH_ALLOCATOR_FUNC_DEFINE(table, ObSimpleTableSchemaV2);
|
||||
} else if (OB_FAIL(schema_array.reserve(schema_keys.count()))) {\
|
||||
LOG_WARN("fail to reserve schema array", KR(ret)); \
|
||||
} else { \
|
||||
std::sort(schema_keys.begin(), schema_keys.end(), SchemaKey::cmp_with_tenant_id); \
|
||||
lib::ob_sort(schema_keys.begin(), schema_keys.end(), SchemaKey::cmp_with_tenant_id); \
|
||||
int64_t begin = 0; \
|
||||
int64_t end = 0; \
|
||||
while (OB_SUCCESS == ret && end < schema_keys.count()) { \
|
||||
@ -3633,7 +3633,7 @@ int ObSchemaServiceSQLImpl::get_batch_tenants(
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
|
||||
std::sort(tenant_ids.begin(), tenant_ids.end());
|
||||
lib::ob_sort(tenant_ids.begin(), tenant_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -3671,7 +3671,7 @@ int ObSchemaServiceSQLImpl::get_batch_synonyms(
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
|
||||
std::sort(tenant_synonym_ids.begin(), tenant_synonym_ids.end());
|
||||
lib::ob_sort(tenant_synonym_ids.begin(), tenant_synonym_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -3815,7 +3815,7 @@ int ObSchemaServiceSQLImpl::get_batch_outlines(
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
|
||||
std::sort(tenant_outline_ids.begin(), tenant_outline_ids.end());
|
||||
lib::ob_sort(tenant_outline_ids.begin(), tenant_outline_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -3853,7 +3853,7 @@ int ObSchemaServiceSQLImpl::get_batch_routines(
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
|
||||
std::sort(tenant_routine_ids.begin(), tenant_routine_ids.end());
|
||||
lib::ob_sort(tenant_routine_ids.begin(), tenant_routine_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -3894,7 +3894,7 @@ int ObSchemaServiceSQLImpl::get_batch_udts(
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
|
||||
std::sort(tenant_udt_ids.begin(), tenant_udt_ids.end());
|
||||
lib::ob_sort(tenant_udt_ids.begin(), tenant_udt_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -3942,7 +3942,7 @@ int ObSchemaServiceSQLImpl::get_batch_users(
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
|
||||
std::sort(tenant_user_ids.begin(), tenant_user_ids.end());
|
||||
lib::ob_sort(tenant_user_ids.begin(), tenant_user_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -4041,7 +4041,7 @@ int ObSchemaServiceSQLImpl::get_batch_packages(const ObRefreshSchemaStatus &sche
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
|
||||
std::sort(tenant_package_ids.begin(), tenant_package_ids.end());
|
||||
lib::ob_sort(tenant_package_ids.begin(), tenant_package_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -4078,7 +4078,7 @@ int ObSchemaServiceSQLImpl::get_batch_mock_fk_parent_tables(
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
std::sort(tenant_mock_fk_parent_table_ids.begin(), tenant_mock_fk_parent_table_ids.end());
|
||||
lib::ob_sort(tenant_mock_fk_parent_table_ids.begin(), tenant_mock_fk_parent_table_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -4115,7 +4115,7 @@ int ObSchemaServiceSQLImpl::get_batch_triggers(const ObRefreshSchemaStatus &sche
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
|
||||
std::sort(tenant_trigger_ids.begin(), tenant_trigger_ids.end());
|
||||
lib::ob_sort(tenant_trigger_ids.begin(), tenant_trigger_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -7536,7 +7536,7 @@ int ObSchemaServiceSQLImpl::get_batch_sequences(
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
|
||||
std::sort(tenant_sequence_ids.begin(), tenant_sequence_ids.end());
|
||||
lib::ob_sort(tenant_sequence_ids.begin(), tenant_sequence_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -7633,7 +7633,7 @@ int ObSchemaServiceSQLImpl::get_batch_label_se_policys(
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
std::sort(tenant_label_se_policy_ids.begin(), tenant_label_se_policy_ids.end());
|
||||
lib::ob_sort(tenant_label_se_policy_ids.begin(), tenant_label_se_policy_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -7728,7 +7728,7 @@ int ObSchemaServiceSQLImpl::get_batch_profiles(
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
std::sort(tenant_profile_ids.begin(), tenant_profile_ids.end());
|
||||
lib::ob_sort(tenant_profile_ids.begin(), tenant_profile_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -7824,7 +7824,7 @@ int ObSchemaServiceSQLImpl::get_batch_label_se_components(
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
std::sort(tenant_label_se_component_ids.begin(), tenant_label_se_component_ids.end());
|
||||
lib::ob_sort(tenant_label_se_component_ids.begin(), tenant_label_se_component_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -7918,7 +7918,7 @@ int ObSchemaServiceSQLImpl::get_batch_label_se_labels(
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
std::sort(tenant_label_se_label_ids.begin(), tenant_label_se_label_ids.end());
|
||||
lib::ob_sort(tenant_label_se_label_ids.begin(), tenant_label_se_label_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -8012,7 +8012,7 @@ int ObSchemaServiceSQLImpl::get_batch_label_se_user_levels(
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
std::sort(tenant_label_se_user_level_ids.begin(), tenant_label_se_user_level_ids.end());
|
||||
lib::ob_sort(tenant_label_se_user_level_ids.begin(), tenant_label_se_user_level_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -8108,7 +8108,7 @@ int ObSchemaServiceSQLImpl::get_batch_tablespaces(
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
std::sort(tenant_tablespace_ids.begin(), tenant_tablespace_ids.end());
|
||||
lib::ob_sort(tenant_tablespace_ids.begin(), tenant_tablespace_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -8244,7 +8244,7 @@ int ObSchemaServiceSQLImpl::get_batch_udfs(
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
|
||||
std::sort(tenant_udf_ids.begin(), tenant_udf_ids.end());
|
||||
lib::ob_sort(tenant_udf_ids.begin(), tenant_udf_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -8342,7 +8342,7 @@ int ObSchemaServiceSQLImpl::get_batch_keystores(
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
std::sort(tenant_keystore_ids.begin(), tenant_keystore_ids.end());
|
||||
lib::ob_sort(tenant_keystore_ids.begin(), tenant_keystore_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -8438,7 +8438,7 @@ int ObSchemaServiceSQLImpl::get_batch_audits(
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("check inner stat fail");
|
||||
}
|
||||
std::sort(tenant_audit_ids.begin(), tenant_audit_ids.end());
|
||||
lib::ob_sort(tenant_audit_ids.begin(), tenant_audit_ids.end());
|
||||
// split query to tenant space && split big query
|
||||
int64_t begin = 0;
|
||||
int64_t end = 0;
|
||||
@ -8903,24 +8903,24 @@ int ObSchemaServiceSQLImpl::sort_partition_array(ObPartitionSchema &partition_sc
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("partition array is empty", K(ret), K(partition_schema));
|
||||
} else if (partition_schema.is_range_part()) {
|
||||
std::sort(partition_schema.get_part_array(),
|
||||
lib::ob_sort(partition_schema.get_part_array(),
|
||||
partition_schema.get_part_array() + partition_num,
|
||||
ObBasePartition::range_like_func_less_than);
|
||||
std::sort(partition_schema.get_hidden_part_array(),
|
||||
lib::ob_sort(partition_schema.get_hidden_part_array(),
|
||||
partition_schema.get_hidden_part_array() + partition_schema.get_hidden_partition_num(),
|
||||
ObBasePartition::range_like_func_less_than);
|
||||
} else if (partition_schema.is_hash_like_part()) {
|
||||
std::sort(partition_schema.get_part_array(),
|
||||
lib::ob_sort(partition_schema.get_part_array(),
|
||||
partition_schema.get_part_array() + partition_num,
|
||||
ObBasePartition::hash_like_func_less_than);
|
||||
std::sort(partition_schema.get_hidden_part_array(),
|
||||
lib::ob_sort(partition_schema.get_hidden_part_array(),
|
||||
partition_schema.get_hidden_part_array() + partition_schema.get_hidden_partition_num(),
|
||||
ObBasePartition::hash_like_func_less_than);
|
||||
} else if (partition_schema.is_list_part()) {
|
||||
std::sort(partition_schema.get_part_array(),
|
||||
lib::ob_sort(partition_schema.get_part_array(),
|
||||
partition_schema.get_part_array() + partition_num,
|
||||
ObBasePartition::list_part_func_layout);
|
||||
std::sort(partition_schema.get_hidden_part_array(),
|
||||
lib::ob_sort(partition_schema.get_hidden_part_array(),
|
||||
partition_schema.get_hidden_part_array() + partition_schema.get_hidden_partition_num(),
|
||||
ObBasePartition::list_part_func_layout);
|
||||
} else {
|
||||
@ -8961,20 +8961,20 @@ int ObSchemaServiceSQLImpl::sort_subpartition_array(ObPartitionSchema &partition
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_ERROR("subpartition array is empty", K(ret), K(i), K(partition_schema));
|
||||
} else if (partition_schema.is_range_subpart()) {
|
||||
std::sort(subpart_array, subpart_array + subpartition_num, ObBasePartition::less_than);
|
||||
std::sort(partition->get_hidden_subpart_array(),
|
||||
lib::ob_sort(subpart_array, subpart_array + subpartition_num, ObBasePartition::less_than);
|
||||
lib::ob_sort(partition->get_hidden_subpart_array(),
|
||||
partition->get_hidden_subpart_array() + partition->get_hidden_subpartition_num(),
|
||||
ObBasePartition::less_than);
|
||||
} else if (partition_schema.is_hash_like_subpart()) {
|
||||
std::sort(subpart_array, subpart_array + subpartition_num,
|
||||
lib::ob_sort(subpart_array, subpart_array + subpartition_num,
|
||||
ObSubPartition::hash_like_func_less_than);
|
||||
std::sort(partition->get_hidden_subpart_array(),
|
||||
lib::ob_sort(partition->get_hidden_subpart_array(),
|
||||
partition->get_hidden_subpart_array() + partition->get_hidden_subpartition_num(),
|
||||
ObSubPartition::hash_like_func_less_than);
|
||||
} else if (partition_schema.is_list_subpart()) {
|
||||
std::sort(subpart_array, subpart_array + subpartition_num,
|
||||
lib::ob_sort(subpart_array, subpart_array + subpartition_num,
|
||||
ObBasePartition::list_part_func_layout);
|
||||
std::sort(partition->get_hidden_subpart_array(),
|
||||
lib::ob_sort(partition->get_hidden_subpart_array(),
|
||||
partition->get_hidden_subpart_array() + partition->get_hidden_subpartition_num(),
|
||||
ObBasePartition::list_part_func_layout);
|
||||
}
|
||||
@ -8995,15 +8995,15 @@ int ObSchemaServiceSQLImpl::sort_subpartition_array(ObPartitionSchema &partition
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("def_subpartition array is empty", K(ret), K(partition_schema));
|
||||
} else if (partition_schema.is_range_subpart()) {
|
||||
std::sort(partition_schema.get_def_subpart_array(),
|
||||
lib::ob_sort(partition_schema.get_def_subpart_array(),
|
||||
partition_schema.get_def_subpart_array() + def_subpartition_num,
|
||||
ObBasePartition::less_than);
|
||||
} else if (partition_schema.is_hash_like_subpart()) {
|
||||
std::sort(partition_schema.get_def_subpart_array(),
|
||||
lib::ob_sort(partition_schema.get_def_subpart_array(),
|
||||
partition_schema.get_def_subpart_array() + def_subpartition_num,
|
||||
ObSubPartition::hash_like_func_less_than);
|
||||
} else if (partition_schema.is_list_subpart()) {
|
||||
std::sort(partition_schema.get_def_subpart_array(),
|
||||
lib::ob_sort(partition_schema.get_def_subpart_array(),
|
||||
partition_schema.get_def_subpart_array() + def_subpartition_num,
|
||||
ObBasePartition::list_part_func_layout);
|
||||
} else {
|
||||
|
@ -5599,7 +5599,7 @@ int ObBasePartition::set_list_vector_values_with_hex_str(
|
||||
// Sort each point in the partition list according to the rules
|
||||
if (OB_SUCC(ret)) {
|
||||
InnerPartListVectorCmp part_list_vector_op;
|
||||
std::sort(list_row_values_.begin(), list_row_values_.end(), part_list_vector_op);
|
||||
lib::ob_sort(list_row_values_.begin(), list_row_values_.end(), part_list_vector_op);
|
||||
if (OB_FAIL(part_list_vector_op.get_ret())) {
|
||||
LOG_WARN("fail to sort list row values", K(ret));
|
||||
}
|
||||
|
@ -3831,7 +3831,7 @@ int ObTableSchema::sort_column_array_by_column_id()
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(column_array_, column_array_ + column_cnt_, [](ObColumnSchemaV2 *&lhs, ObColumnSchemaV2 *&rhs) -> bool {
|
||||
lib::ob_sort(column_array_, column_array_ + column_cnt_, [](ObColumnSchemaV2 *&lhs, ObColumnSchemaV2 *&rhs) -> bool {
|
||||
return lhs->get_column_id() < rhs->get_column_id();
|
||||
});
|
||||
}
|
||||
|
@ -346,7 +346,7 @@ int ObStatTopKHist::build_histogram_from_topk_items(ObIAllocator &allocator,
|
||||
}
|
||||
if (OB_SUCC(ret) && tmp.count() > 0) {
|
||||
ObBucketCompare cmp;
|
||||
std::sort(&tmp.at(0),
|
||||
lib::ob_sort(&tmp.at(0),
|
||||
&tmp.at(0) + tmp.count(),
|
||||
CopyableBucketComparer(cmp));
|
||||
if (OB_FAIL(cmp.ret_)) {
|
||||
|
@ -106,7 +106,7 @@ int ObTopKFrequencyHistograms::sort_topk_fre_items(ObIArray<ObTopkItem> &items)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
Compare cmp;
|
||||
std::sort(&items.at(0),
|
||||
lib::ob_sort(&items.at(0),
|
||||
&items.at(0)+ items.count(),
|
||||
CopyableComparer(cmp));
|
||||
if (OB_FAIL(cmp.ret_)) {
|
||||
|
@ -2694,7 +2694,7 @@ int ObDmlCgService::convert_normal_triggers(ObLogDelUpd &log_op,
|
||||
}
|
||||
trig_ctdef.tg_event_ = dml_event;
|
||||
ObTriggerInfo::ActionOrderComparator action_order_com;
|
||||
std::sort(trigger_infos.begin(), trigger_infos.end(), action_order_com);
|
||||
lib::ob_sort(trigger_infos.begin(), trigger_infos.end(), action_order_com);
|
||||
if (OB_FAIL(action_order_com.get_ret())) {
|
||||
ret = common::OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("sort error", K(ret));
|
||||
|
@ -1960,7 +1960,7 @@ int ObWhiteFilterExecutor::init_in_eval_datums()
|
||||
if (OB_SUCC(ret)) {
|
||||
bool mock_equal = false;
|
||||
ObDatumComparator cmp(filter_.expr_->args_[1]->args_[0]->basic_funcs_->null_first_cmp_, ret, mock_equal);
|
||||
std::sort(datum_params_.begin(), datum_params_.end(), cmp);
|
||||
lib::ob_sort(datum_params_.begin(), datum_params_.end(), cmp);
|
||||
if (OB_FAIL(ret)) {
|
||||
LOG_WARN("Failed to sort datums", K(ret));
|
||||
} else {
|
||||
|
@ -378,7 +378,7 @@ int ObPartitionExecutorUtils::cast_list_expr_to_obj(
|
||||
? subpartition_array[i]->list_row_values_
|
||||
: partition_array[i]->list_row_values_;
|
||||
InnerPartListVectorCmp part_list_vector_op;
|
||||
std::sort(list_row_values.begin(), list_row_values.end(), part_list_vector_op);
|
||||
lib::ob_sort(list_row_values.begin(), list_row_values.end(), part_list_vector_op);
|
||||
if (OB_FAIL(part_list_vector_op.get_ret())) {
|
||||
LOG_WARN("fail to sort list row values", K(ret));
|
||||
}
|
||||
@ -392,7 +392,7 @@ int ObPartitionExecutorUtils::cast_list_expr_to_obj(
|
||||
// 不在这里排序一级list分区是因为对于非模板化二级分区,排序后会导致partition array与
|
||||
// individual_subpart_values_exprs对应不上。
|
||||
if (is_subpart) {
|
||||
std::sort(subpartition_array,
|
||||
lib::ob_sort(subpartition_array,
|
||||
subpartition_array + array_count,
|
||||
ObBasePartition::list_part_func_layout);
|
||||
}
|
||||
@ -1206,7 +1206,7 @@ int ObPartitionExecutorUtils::cast_list_expr_to_obj(
|
||||
? subpartition_array[i]->list_row_values_
|
||||
: partition_array[i]->list_row_values_;
|
||||
InnerPartListVectorCmp part_list_vector_op;
|
||||
std::sort(list_row_values.begin(), list_row_values.end(), part_list_vector_op);
|
||||
lib::ob_sort(list_row_values.begin(), list_row_values.end(), part_list_vector_op);
|
||||
if (OB_FAIL(part_list_vector_op.get_ret())) {
|
||||
LOG_WARN("fail to sort list row values", K(ret));
|
||||
}
|
||||
@ -1219,11 +1219,11 @@ int ObPartitionExecutorUtils::cast_list_expr_to_obj(
|
||||
const int64_t array_count = list_values_exprs.count();
|
||||
// TODO(yibo) tablegroup 支持二级分区异构后,一级分区的排序也要延迟处理
|
||||
if (is_subpart) {
|
||||
std::sort(subpartition_array,
|
||||
lib::ob_sort(subpartition_array,
|
||||
subpartition_array + array_count,
|
||||
ObBasePartition::list_part_func_layout);
|
||||
} else {
|
||||
std::sort(partition_array,
|
||||
lib::ob_sort(partition_array,
|
||||
partition_array + array_count,
|
||||
ObBasePartition::list_part_func_layout);
|
||||
}
|
||||
@ -1426,7 +1426,7 @@ int ObPartitionExecutorUtils::sort_list_paritition_if_need(ObTableSchema &table_
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret));
|
||||
} else {
|
||||
std::sort(partition_array,
|
||||
lib::ob_sort(partition_array,
|
||||
partition_array + array_count,
|
||||
ObBasePartition::list_part_func_layout);
|
||||
}
|
||||
|
@ -188,9 +188,9 @@ int ObExprCollPred::calc_collection_is_contained_without_null(const pl::ObPLColl
|
||||
result = COLL_PRED_FALSE;
|
||||
} else {
|
||||
const ObObj **first = &c1_copy.at(0);
|
||||
std::sort(first, first + c1_cnt, uc);
|
||||
lib::ob_sort(first, first + c1_cnt, uc);
|
||||
first = &c2_copy.at(0);
|
||||
std::sort(first, first + c2_cnt, uc);
|
||||
lib::ob_sort(first, first + c2_cnt, uc);
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < c1_copy.count(); ++i) {
|
||||
if (0 != compare_obj(*(c1_copy.at(i)), *(c2_copy.at(i)), cmp_ctx)) {
|
||||
result = COLL_PRED_FALSE;
|
||||
|
@ -517,7 +517,7 @@ int ObExprJsonQuery::set_multivalue_result(ObEvalCtx& ctx,
|
||||
if (OB_SUCC(ret) && OB_NOT_NULL(obj_array)) {
|
||||
lib::ObMallocHookAttrGuard malloc_guard(lib::ObMemAttr(ObXMLExprHelper::get_tenant_id(ctx.exec_ctx_.get_my_session()), "JSONMultivalue"));
|
||||
ObJsonObjectCompare cmp;
|
||||
std::sort(obj_array, obj_array + element_count, cmp);
|
||||
lib::ob_sort(obj_array, obj_array + element_count, cmp);
|
||||
}
|
||||
|
||||
uint32_t real_store_count = 0;
|
||||
|
@ -3045,9 +3045,9 @@ int ObRelationalExprOperator::pl_udt_compare2(CollectionPredRes &cmp_result,
|
||||
};
|
||||
udtComparer uc(cmp_ctx, lt_cmp_fp);
|
||||
const ObObj **first = &c1_copy.at(0);
|
||||
std::sort(first, first + c1_copy.count(), uc);
|
||||
lib::ob_sort(first, first + c1_copy.count(), uc);
|
||||
first = &c2_copy.at(0);
|
||||
std::sort(first, first + c2_copy.count(), uc);
|
||||
lib::ob_sort(first, first + c2_copy.count(), uc);
|
||||
int cmp_res = 1;
|
||||
// 可能是等值或者不等值
|
||||
common::obj_cmp_func eq_cmp_fp;
|
||||
|
@ -98,7 +98,7 @@ int ObPxMultiPartSSTableInsertOp::inner_open()
|
||||
LOG_WARN("get snapshot version failed", K(ret));
|
||||
} else {
|
||||
// sort in ASC order by tablet id.
|
||||
std::sort(participants_.begin(), participants_.end(), ObLSTabletIDPairCmp());
|
||||
lib::ob_sort(participants_.begin(), participants_.end(), ObLSTabletIDPairCmp());
|
||||
op_monitor_info_.otherstat_1_id_ = ObSqlMonitorStatIds::SSTABLE_INSERT_CG_ROW_COUNT;
|
||||
op_monitor_info_.otherstat_1_value_ = 0;
|
||||
op_monitor_info_.otherstat_2_id_ = ObSqlMonitorStatIds::SSTABLE_INSERT_ROW_COUNT;
|
||||
@ -386,7 +386,7 @@ int ObPxMultiPartSSTableInsertOp::get_all_rows_and_count()
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(tablet_seq_caches_.begin(), tablet_seq_caches_.end());
|
||||
lib::ob_sort(tablet_seq_caches_.begin(), tablet_seq_caches_.end());
|
||||
count_rows_finish_ = true;
|
||||
}
|
||||
}
|
||||
|
@ -263,7 +263,7 @@ int ObRDWFPieceMsgCtx::send_whole_msg(common::ObIArray<ObPxSqcMeta *> &sqcs)
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
// order by sqc_id_, thread_id_
|
||||
std::sort(infos_.begin(), infos_.end(), [](ObRDWFPartialInfo *l,
|
||||
lib::ob_sort(infos_.begin(), infos_.end(), [](ObRDWFPartialInfo *l,
|
||||
ObRDWFPartialInfo *r) {
|
||||
return std::tie(l->sqc_id_, l->thread_id_) < std::tie(r->sqc_id_, r->thread_id_);
|
||||
});
|
||||
|
@ -312,7 +312,7 @@ int ObPxDistTransmitOp::do_bc2host_dist()
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(channel_idx.begin(), channel_idx.end(), [&channels](int64_t l, int64_t r) {
|
||||
lib::ob_sort(channel_idx.begin(), channel_idx.end(), [&channels](int64_t l, int64_t r) {
|
||||
return channels.at(l)->get_peer() < channels.at(r)->get_peer(); });
|
||||
}
|
||||
ObBc2HostSliceIdCalc::HostIndex hi;
|
||||
@ -965,7 +965,7 @@ int ObPxDistTransmitOp::setup_sampled_rows_output()
|
||||
OZ(sampled_rows2transmit_.push_back(std::make_pair(i, nrand48(rand48_buf_))));
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
std::sort(sampled_rows2transmit_.begin(), sampled_rows2transmit_.end(),
|
||||
lib::ob_sort(sampled_rows2transmit_.begin(), sampled_rows2transmit_.end(),
|
||||
[](const std::pair<int64_t, int64_t> &l, const std::pair<int64_t, int64_t> &r)
|
||||
{ return l.second < r.second; });
|
||||
FOREACH(range, sampled_rows2transmit_) {
|
||||
|
@ -260,7 +260,7 @@ int ObGITaskSet::construct_taskset(ObIArray<ObDASTabletLoc*> &taskset_tablets,
|
||||
}
|
||||
if (OB_SUCC(ret) && random_type != GI_RANDOM_NONE) {
|
||||
auto compare_fun = [](const ObGITaskInfo &a, const ObGITaskInfo &b) -> bool { return a.hash_value_ > b.hash_value_; };
|
||||
std::sort(gi_task_set_.begin(), gi_task_set_.end(), compare_fun);
|
||||
lib::ob_sort(gi_task_set_.begin(), gi_task_set_.end(), compare_fun);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -900,7 +900,7 @@ int ObGranuleSplitter::get_query_range(ObExecContext &ctx,
|
||||
(some scenarios do not require order), but the logic is acceptable.
|
||||
2. If reverse order is required outside, then the reverse sorting should be done outside on its own.
|
||||
*/
|
||||
std::sort(scan_ranges.begin(), scan_ranges.end(), ObNewRangeCmp());
|
||||
lib::ob_sort(scan_ranges.begin(), scan_ranges.end(), ObNewRangeCmp());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -111,7 +111,7 @@ int ObPXServerAddrUtil::sort_and_collect_local_file_distribution(
|
||||
auto addrcmp = [](const ObExternalFileInfo &l, const ObExternalFileInfo &r) -> bool {
|
||||
return l.file_addr_ < r.file_addr_;
|
||||
};
|
||||
std::sort(files.get_data(), files.get_data() + files.count(), addrcmp);
|
||||
lib::ob_sort(files.get_data(), files.get_data() + files.count(), addrcmp);
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < files.count(); i++) {
|
||||
ObAddr &cur_addr = files.at(i).file_addr_;
|
||||
@ -1249,7 +1249,7 @@ int ObPXServerAddrUtil::reorder_all_partitions(int64_t table_location_key,
|
||||
if (OB_SUCC(ret)) {
|
||||
try {
|
||||
if (!is_virtual_table(ref_table_id)) {
|
||||
std::sort(&dst_locations.at(0),
|
||||
lib::ob_sort(&dst_locations.at(0),
|
||||
&dst_locations.at(0) + dst_locations.count(),
|
||||
ObPXTabletOrderIndexCmp(asc, &tablet_order_map));
|
||||
}
|
||||
@ -1360,7 +1360,7 @@ int ObPXServerAddrUtil::split_parallel_into_task(const int64_t parallel,
|
||||
}
|
||||
// 排序,执行时间长的排在前面
|
||||
auto compare_fun_long_time_first = [](ObPxSqcTaskCountMeta a, ObPxSqcTaskCountMeta b) -> bool { return a.time_ > b.time_; };
|
||||
std::sort(sqc_task_metas.begin(),
|
||||
lib::ob_sort(sqc_task_metas.begin(),
|
||||
sqc_task_metas.end(),
|
||||
compare_fun_long_time_first);
|
||||
/// 把剩下的线程安排出去
|
||||
@ -2734,7 +2734,7 @@ int ObPxAffinityByRandom::do_random(bool use_partition_info, uint64_t tenant_id)
|
||||
// So we sort partitions by partition_idx and generate a relative_idx which starts from zero.
|
||||
// Then calculate hash value with the relative_idx
|
||||
auto part_idx_compare_fun = [](TabletHashValue a, TabletHashValue b) -> bool { return a.tablet_idx_ > b.tablet_idx_; };
|
||||
std::sort(tablet_hash_values_.begin(),
|
||||
lib::ob_sort(tablet_hash_values_.begin(),
|
||||
tablet_hash_values_.end(),
|
||||
part_idx_compare_fun);
|
||||
int64_t relative_idx = 0;
|
||||
@ -2747,7 +2747,7 @@ int ObPxAffinityByRandom::do_random(bool use_partition_info, uint64_t tenant_id)
|
||||
|
||||
// 先打乱所有的序
|
||||
auto compare_fun = [](TabletHashValue a, TabletHashValue b) -> bool { return a.hash_value_ > b.hash_value_; };
|
||||
std::sort(tablet_hash_values_.begin(),
|
||||
lib::ob_sort(tablet_hash_values_.begin(),
|
||||
tablet_hash_values_.end(),
|
||||
compare_fun);
|
||||
LOG_TRACE("after sort partition_hash_values randomly", K(tablet_hash_values_), K(this), K(order_partitions_));
|
||||
@ -2780,12 +2780,12 @@ int ObPxAffinityByRandom::do_random(bool use_partition_info, uint64_t tenant_id)
|
||||
// 保持序
|
||||
if (asc_order) {
|
||||
auto compare_fun_order_by_part_asc = [](TabletHashValue a, TabletHashValue b) -> bool { return a.tablet_idx_ < b.tablet_idx_; };
|
||||
std::sort(tablet_hash_values_.begin(),
|
||||
lib::ob_sort(tablet_hash_values_.begin(),
|
||||
tablet_hash_values_.end(),
|
||||
compare_fun_order_by_part_asc);
|
||||
} else {
|
||||
auto compare_fun_order_by_part_desc = [](TabletHashValue a, TabletHashValue b) -> bool { return a.tablet_idx_ > b.tablet_idx_; };
|
||||
std::sort(tablet_hash_values_.begin(),
|
||||
lib::ob_sort(tablet_hash_values_.begin(),
|
||||
tablet_hash_values_.end(),
|
||||
compare_fun_order_by_part_desc);
|
||||
}
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
x
Reference in New Issue
Block a user