cherry-pick bug fix in 3.1

This commit is contained in:
bf0
2021-07-21 23:46:35 +08:00
committed by wangzelin.wzl
parent d49f1baf7d
commit 7fb21bbbe7
23 changed files with 450 additions and 208 deletions

View File

@ -727,8 +727,11 @@ int ObLoadDataImpl::take_record_for_failed_rows(ObPhysicalPlanCtx& plan_ctx, ObL
return ret;
}
int ObLoadDataBase::memory_wait_local(
ObExecContext& ctx, const ObPartitionKey& part_key, ObAddr& server_addr, int64_t& total_wait_secs)
int ObLoadDataBase::memory_wait_local(ObExecContext &ctx,
const ObPartitionKey &part_key,
ObAddr &server_addr,
int64_t &total_wait_secs,
bool &is_leader_changed)
{
int ret = OB_SUCCESS;
static const int64_t WAIT_INTERVAL_US = 1 * 1000 * 1000; // 1s
@ -811,6 +814,9 @@ int ObLoadDataBase::memory_wait_local(
if (leader_addr != server_addr) {
LOG_INFO("LOAD DATA location change", K(part_key), "old_addr", server_addr, "new_addr", leader_addr);
server_addr = leader_addr;
is_leader_changed = true;
} else {
is_leader_changed = false;
}
LOG_INFO("LOAD DATA is resumed", "waited_seconds", wait_secs, K(total_wait_secs));
}
@ -3180,12 +3186,22 @@ int ObLoadDataSPImpl::handle_returned_insert_task(
ObAddr& addr = part_mgr->get_leader_addr();
bool found = (OB_SUCCESS == box.server_last_available_ts.get(addr, last_ts));
if (insert_task.result_recv_ts_ > last_ts) {
if (OB_FAIL(memory_wait_local(ctx, part_mgr->get_part_key(), addr, box.wait_secs_for_mem_release))) {
bool is_leader_changed = false;
if (OB_FAIL(memory_wait_local(ctx, part_mgr->get_part_key(),
addr, box.wait_secs_for_mem_release,
is_leader_changed))) {
LOG_WARN("fail to memory_wait_local", K(ret));
} else {
int64_t curr_time = ObTimeUtil::current_time();
if (is_leader_changed) {
found = (OB_SUCCESS == box.server_last_available_ts.get(addr, last_ts));
}
ret = found ? box.server_last_available_ts.update(addr, curr_time)
: box.server_last_available_ts.insert(addr, curr_time);
if (OB_FAIL(ret)) {
LOG_WARN("failt to update server_last_available_ts",
K(ret), K(addr), K(found), K(is_leader_changed));
}
}
}
}
@ -3214,45 +3230,41 @@ int ObLoadDataSPImpl::handle_returned_insert_task(
if (OB_SUCC(ret)) {
switch (task_status) {
case TASK_SUCC:
box.affected_rows += insert_task.row_count_;
box.insert_rt_sum += insert_task.process_us_;
/* RESERVE FOR DEBUG
box.handle_returned_insert_task_count++;
if (insert_task.row_count_ != DEFAULT_BUFFERRED_ROW_COUNT) {
LOG_WARN("LOAD DATA task return",
"task_id", insert_task.task_id_,
"affected_rows", box.affected_rows,
"row_count", insert_task.row_count_);
}
*/
break;
case TASK_NEED_RETRY:
insert_task.retry_times_++;
need_retry = true;
LOG_WARN("LOAD DATA task need retry",
"task_id",
insert_task.task_id_,
"ret",
result.exec_ret_,
"row_count",
insert_task.row_count_);
break;
case TASK_FAILED:
if (OB_SUCCESS != log_failed_insert_task(box, insert_task)) {
LOG_WARN("fail to log failed insert task");
}
LOG_WARN("LOAD DATA task failed",
"task_id",
insert_task.task_id_,
"ret",
result.exec_ret_,
"row_count",
insert_task.row_count_);
break;
default:
ret = OB_ERR_UNEXPECTED;
break;
case TASK_SUCC:
box.affected_rows += insert_task.row_count_;
box.insert_rt_sum += insert_task.process_us_;
/* RESERVE FOR DEBUG
box.handle_returned_insert_task_count++;
if (insert_task.row_count_ != DEFAULT_BUFFERRED_ROW_COUNT) {
LOG_WARN("LOAD DATA task return",
"task_id", insert_task.task_id_,
"affected_rows", box.affected_rows,
"row_count", insert_task.row_count_);
}
*/
break;
case TASK_NEED_RETRY:
insert_task.retry_times_++;
need_retry = true;
LOG_WARN("LOAD DATA task need retry",
"execute server", server_info->addr,
"task_id", insert_task.task_id_,
"ret", result.exec_ret_,
"row_count", insert_task.row_count_);
break;
case TASK_FAILED:
if (OB_SUCCESS != log_failed_insert_task(box, insert_task)) {
LOG_WARN("fail to log failed insert task");
}
LOG_WARN("LOAD DATA task failed",
"execute server", server_info->addr,
"task_id", insert_task.task_id_,
"ret", result.exec_ret_,
"row_count", insert_task.row_count_);
break;
default:
ret = OB_ERR_UNEXPECTED;
break;
}
}
@ -3429,6 +3441,10 @@ int ObLoadDataSPImpl::insert_task_gen_and_dispatch(ObExecContext& ctx, ToolBox&
LOG_WARN("fail to on task finish", K(ret));
} else if (OB_FAIL(box.insert_task_reserve_queue.push_back(insert_task))) {
LOG_WARN("fail to push back", K(ret));
} else if (OB_ISNULL(insert_task)) {
ret = OB_ERR_UNEXPECTED;
} else {
insert_task->reuse();
}
}
@ -4136,11 +4152,11 @@ int ObLoadDataSPImpl::ToolBox::init(ObExecContext& ctx, ObLoadDataStmt& load_stm
if (OB_SUCC(ret)) {
if (OB_FAIL(shuffle_task_controller.init(parallel))) {
LOG_WARN("fail to init shuffle task controller", K(ret));
} else if (OB_FAIL(shuffle_task_reserve_queue.init(parallel))) {
} else if (OB_FAIL(shuffle_task_reserve_queue.init(parallel + 1))) {
LOG_WARN("fail to init shuffle_task_reserve_queue", K(ret));
} else if (OB_FAIL(insert_task_controller.init(parallel * server_infos.count()))) {
LOG_WARN("fail to init insert task controller", K(ret));
} else if (OB_FAIL(insert_task_reserve_queue.init(parallel * server_infos.count()))) {
} else if (OB_FAIL(insert_task_reserve_queue.init(parallel * server_infos.count() + 1))) {
LOG_WARN("fail to init insert_task_reserve_queue", K(ret));
} else if (OB_FAIL(ctx_allocators.reserve(parallel))) {
LOG_WARN("fail to pre alloc allocators", K(ret));

View File

@ -732,7 +732,8 @@ public:
static int memory_check_remote(uint64_t tenant_id, bool& need_wait_minor_freeze);
static int memory_wait_local(
ObExecContext& ctx, const ObPartitionKey& part_key, ObAddr& server_addr, int64_t& total_wait_secs);
ObExecContext& ctx, const ObPartitionKey& part_key, ObAddr& server_addr,
int64_t& total_wait_secs, bool &is_leader_changed);
virtual int execute(ObExecContext& ctx, ObLoadDataStmt& load_stmt) = 0;
};

View File

@ -138,11 +138,19 @@ int ObRpcLoadDataShuffleTaskExecuteP::process()
LOG_WARN("LOAD DATA shuffle task timeout", K(ret), K(task));
} else if (OB_FAIL(task.shuffle_task_handle_.get_arg(handle))) { // check identifier
LOG_ERROR("fail to get arg", K(ret));
} else if (OB_FAIL(ObLoadDataSPImpl::exec_shuffle(task.task_id_, handle))) {
LOG_WARN("fail to exec shuffle task", K(ret));
} else if (OB_ISNULL(handle)) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("handle is null", K(ret));
} else {
if (OB_UNLIKELY(THIS_WORKER.is_timeout())) {
ret = OB_TIMEOUT;
LOG_WARN("LOAD DATA shuffle task timeout", K(ret), K(task));
} else if (OB_FAIL(ObLoadDataSPImpl::exec_shuffle(task.task_id_, handle))) {
LOG_WARN("fail to exec shuffle task", K(ret));
}
handle->result.exec_ret_ = ret;
}
handle->result.exec_ret_ = ret;
MEM_BARRIER(); // use handle ptr before release job ref
MEM_BARRIER(); //use handle ptr before release job ref
job_status->release();
}

View File

@ -22,6 +22,7 @@
#include "sql/engine/cmd/ob_load_data_utils.h"
#include "share/config/ob_server_config.h"
#include "observer/ob_server_struct.h"
#include "lib/lock/ob_spin_lock.h"
namespace oceanbase {
namespace observer {
@ -279,7 +280,7 @@ public:
~ObConcurrentFixedCircularArray()
{
if (data_ != NULL) {
ob_free_align(static_cast<void*>(data_));
ob_free_align((void *)(data_));
}
}
int init(int64_t array_size)
@ -294,8 +295,8 @@ public:
}
return ret;
}
OB_INLINE int push_back(const T& obj)
{
OB_INLINE int push_back(const T &obj) {
common::ObSpinLockGuard guard(lock_);
int ret = common::OB_SUCCESS;
// push optimistically
int64_t pos = ATOMIC_FAA(&head_pos_, 1);
@ -303,12 +304,13 @@ public:
if (OB_UNLIKELY(pos - ATOMIC_LOAD(&tail_pos_) >= array_size_)) {
ret = common::OB_SIZE_OVERFLOW;
} else {
ATOMIC_SET(&data_[pos % array_size_], obj);
ATOMIC_STORE(&data_[pos % array_size_], obj);
}
return ret;
}
OB_INLINE int pop(T& output)
{
common::ObSpinLockGuard guard(lock_);
int ret = common::OB_SUCCESS;
// pop optimistically
int64_t pos = ATOMIC_FAA(&tail_pos_, 1);
@ -316,7 +318,8 @@ public:
if (OB_UNLIKELY(pos >= ATOMIC_LOAD(&head_pos_))) {
ret = common::OB_ARRAY_OUT_OF_RANGE;
} else {
output = ATOMIC_LOAD(&data_[pos % array_size_]);
//output = ATOMIC_LOAD(&data_[pos % array_size_]);
output = ATOMIC_SET(&data_[pos % array_size_], NULL);
}
return ret;
}
@ -328,9 +331,10 @@ public:
private:
// data members
int64_t array_size_;
T* volatile data_;
volatile T * data_;
volatile int64_t head_pos_;
volatile int64_t tail_pos_;
common::ObSpinLock lock_;
};
typedef ObConcurrentFixedCircularArray<ObLoadbuffer*> CompleteTaskArray;

View File

@ -176,6 +176,8 @@
#include "ob_expr_format.h"
#include "ob_expr_quarter.h"
#include "ob_expr_bit_length.h"
#include "ob_expr_to_single_byte.h"
#include "ob_expr_to_multi_byte.h"
namespace oceanbase {
using namespace common;
@ -645,8 +647,8 @@ static ObExpr::EvalFunc g_expr_eval_functions[] = {
NULL, // ObExprAsciistr::calc_asciistr_expr, /* 393 */
NULL, // ObExprAtTimeZone::eval_at_time_zone, /* 394 */
NULL, // ObExprAtLocal::eval_at_local, /* 395 */
NULL, // ObExprToSingleByte::calc_to_single_byte, /* 396 */
NULL, // ObExprToMultiByte::calc_to_multi_byte, /* 397 */
ObExprToSingleByte::calc_to_single_byte, /* 396 */
ObExprToMultiByte::calc_to_multi_byte, /* 397 */
NULL, // ObExprDllUdf::eval_dll_udf, /* 398 */
NULL, // ObExprRawtonhex::calc_rawtonhex_expr, /* 399 */
ObExprPi::eval_pi /* 400 */

View File

@ -613,9 +613,9 @@ int ObExprResultTypeUtil::deduce_max_string_length_oracle(const ObSQLSessionInfo
if (OB_FAIL(ret)) {
} else if (orig_type.is_character_type()) {
CK(LS_BYTE == orig_ls || LS_CHAR == orig_ls);
if (OB_FAIL(ret)) {
} else if (LS_BYTE == target_ls && LS_CHAR == orig_ls) {
// 当参数为 character 类型是,语义必定为 byte 或 char
if (LS_BYTE == target_ls && LS_CHAR == orig_ls) {
// 当从 char 语义 转化到 byte 语义时,需要乘以 mbmaxlen
length *= mbmaxlen;
} else if (LS_CHAR == target_ls && LS_BYTE == orig_ls) {
ret = OB_ERR_UNEXPECTED;

View File

@ -56,66 +56,125 @@ int ObExprToMultiByte::calc_result_type1(ObExprResType& type, ObExprResType& typ
return ret;
}
int ObExprToMultiByte::calc_result1(ObObj& result, const ObObj& obj, ObExprCtx& expr_ctx) const
int calc_to_multi_byte_expr(const ObString &input, const ObCollationType cs_type,
char *buf, const int64_t buf_len, int32_t &pos)
{
int ret = OB_SUCCESS;
int64_t min_char_width = 0;
if (OB_FAIL(ObCharset::get_mbminlen_by_coll(cs_type, min_char_width))) {
LOG_WARN("fail to get mbminlen", K(ret));
} else if (min_char_width > 1) {
ObDataBuffer allocator(buf, buf_len);
ObString output;
if (OB_FAIL(ob_write_string(allocator, input, output))) {
LOG_WARN("invalid input value", K(ret), K(buf_len), K(input));
} else {
pos = output.length();
}
} else {
ObStringScanner scanner(input, cs_type);
ObString encoding;
int32_t wc = 0;
char *ptr = buf;
while (OB_SUCC(ret)
&& scanner.next_character(encoding, wc, ret)) {
int32_t length = 0;
if (wc == 0x20) { //处理空格
wc = 0x3000;
//smart quote not support https://gerry.lamost.org/blog/?p=295757
//} else if (wc == 0x22) { //" --> ” smart double quote
// wc = 0x201D;
//} else if (wc == 0x27) { //' --> ’ smart single quote
// wc = 0x2019;
} else if (wc >= 0x21 && wc <= 0x7E) {
wc += 65248;
} else {
//do nothing
}
OZ (ObCharset::wc_mb(cs_type, wc, ptr, buf + buf_len - ptr, length));
ptr += length;
LOG_DEBUG("process char", K(ret), K(wc));
}
pos = ptr - buf;
}
return ret;
}
int ObExprToMultiByte::calc_result1(ObObj &result,
const ObObj &obj,
ObExprCtx &expr_ctx) const
{
int ret = OB_SUCCESS;
ObString src_string;
ObString dst_string;
ObCollationType src_collation = obj.get_collation_type();
int64_t min_char_width = 0;
if (obj.is_null_oracle()) {
result.set_null();
} else {
CK(OB_NOT_NULL(expr_ctx.calc_buf_));
OZ(obj.get_string(src_string));
char* buff = NULL;
int64_t buff_len = src_string.length() * 4;
CK (OB_NOT_NULL(expr_ctx.calc_buf_));
OZ (obj.get_string(src_string));
if (OB_SUCC(ret)) {
char* buff = NULL;
int64_t buff_len = src_string.length() * ObCharset::MAX_MB_LEN;
int32_t pos = 0;
if (OB_ISNULL(buff = static_cast<char*>(expr_ctx.calc_buf_->alloc(buff_len)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("fail to allocate buffer", K(ret), K(buff_len));
} else if (OB_FAIL(calc_to_multi_byte_expr(src_string, src_collation, buff, buff_len, pos))) {
LOG_WARN("fail to calc", K(ret));
} else {
dst_string.assign_buffer(buff, buff_len);
result.set_common_value(ObString(pos, buff));
result.set_meta_type(result_type_);
}
}
OZ(ObCharset::get_mbminlen_by_coll(src_collation, min_char_width));
}
return ret;
}
if (min_char_width > 1) {
OX(dst_string.write(src_string.ptr(), src_string.length()));
OX(result.set_nvarchar2(dst_string));
int ObExprToMultiByte::cg_expr(ObExprCGCtx &op_cg_ctx,
const ObRawExpr &raw_expr,
ObExpr &rt_expr) const
{
int ret = OB_SUCCESS;
UNUSED(op_cg_ctx);
UNUSED(raw_expr);
rt_expr.eval_func_ = calc_to_multi_byte;
return ret;
}
int ObExprToMultiByte::calc_to_multi_byte(const ObExpr &expr,
ObEvalCtx &ctx,
ObDatum &res_datum)
{
int ret = OB_SUCCESS;
ObDatum *src_param = NULL;
if (expr.args_[0]->eval(ctx, src_param)) {
LOG_WARN("eval arg failed", K(ret));
} else {
if (src_param->is_null()) {
res_datum.set_null();
} else {
const char* src_ptr = src_string.ptr();
int64_t src_pos = 0;
char* dst_ptr = buff;
int64_t dst_pos = 0;
ObString src = src_param->get_string();
char *buf = NULL;
int64_t buf_len = src.length() * ObCharset::MAX_MB_LEN;
int32_t length = 0;
int32_t wc = 0;
while (OB_SUCC(ret) && src_pos < src_string.length()) {
OZ(ObCharset::mb_wc(src_collation, src_ptr + src_pos, src_string.length() - src_pos, length, wc));
OX(src_pos += length);
int32_t wc_before_process = wc;
if (wc == 0x20) {
wc = 0x3000;
// smart quote not support https://gerry.lamost.org/blog/?p=295757
//} else if (wc == 0x22) { //" --> " smart double quote
// wc = 0x201D;
//} else if (wc == 0x27) { //' --> ' smart single quote
// wc = 0x2019;
} else if (wc >= 0x21 && wc <= 0x7E) {
wc += 65248;
} else {
// do nothing
}
LOG_DEBUG("process char", K(ret), K(src_pos), K(wc_before_process), K(wc));
OZ(ObCharset::wc_mb(src_collation, wc, dst_ptr + dst_pos, buff_len - dst_pos, length));
OX(dst_pos += length);
if (OB_ISNULL(buf = static_cast<char*>(expr.get_str_res_mem(ctx, buf_len)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("fail to allocate memory", K(ret), K(src));
} else if (OB_FAIL(calc_to_multi_byte_expr(src, expr.args_[0]->datum_meta_.cs_type_,
buf, buf_len, length))) {
LOG_WARN("fail to calc unistr", K(ret));
} else {
res_datum.set_string(buf, length);
}
OX(dst_string.assign_ptr(dst_ptr, dst_pos));
OX(result.set_varchar(dst_string));
}
OX(result.set_collation(result_type_));
}
return ret;
}

View File

@ -21,9 +21,15 @@ class ObExprToMultiByte : public ObFuncExprOperator {
public:
explicit ObExprToMultiByte(common::ObIAllocator& alloc);
virtual ~ObExprToMultiByte();
int calc_result_type1(ObExprResType& type, ObExprResType& type1, common::ObExprTypeCtx& type_ctx) const;
int calc_result1(common::ObObj& result, const common::ObObj& obj, common::ObExprCtx& expr_ctx) const;
int calc_result_type1(ObExprResType &type,
ObExprResType &type1,
common::ObExprTypeCtx &type_ctx) const;
int calc_result1(common::ObObj &result,
const common::ObObj &obj,
common::ObExprCtx &expr_ctx) const;
int cg_expr(ObExprCGCtx &op_cg_ctx, const ObRawExpr &raw_expr, ObExpr &rt_expr) const;
static int calc_to_multi_byte(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &res_datum);
private:
DISALLOW_COPY_AND_ASSIGN(ObExprToMultiByte);
};

View File

@ -48,71 +48,129 @@ int ObExprToSingleByte::calc_result_type1(ObExprResType& type, ObExprResType& ty
return ret;
}
int ObExprToSingleByte::calc_result1(ObObj& result, const ObObj& obj, ObExprCtx& expr_ctx) const
int calc_to_single_byte_expr(const ObString &input, const ObCollationType cs_type,
char *buf, const int64_t buf_len, int32_t &pos)
{
int ret = OB_SUCCESS;
int64_t min_char_width = 0;
if (obj.is_null_oracle()) {
result.set_null();
} else {
ObString src_string;
ObString dst_string;
ObCollationType src_collation = obj.get_collation_type();
int64_t min_char_width = 0;
char* buff = NULL;
int64_t buff_len = 0;
if (OB_FAIL(ObCharset::get_mbminlen_by_coll(cs_type, min_char_width))) {
LOG_WARN("fail to get mbminlen", K(ret));
} else if (min_char_width > 1) {
ObDataBuffer allocator(buf, buf_len);
ObString output;
CK(OB_NOT_NULL(expr_ctx.calc_buf_));
OZ(obj.get_string(src_string));
if (OB_SUCC(ret)) {
buff_len = src_string.length();
if (OB_ISNULL(buff = static_cast<char*>(expr_ctx.calc_buf_->alloc(buff_len)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("fail to allocate buffer", K(ret), K(buff_len));
} else {
dst_string.assign_buffer(buff, buff_len);
}
}
OZ(ObCharset::get_mbminlen_by_coll(src_collation, min_char_width));
if (min_char_width > 1) {
OX(dst_string.write(src_string.ptr(), src_string.length()));
if (OB_FAIL(ob_write_string(allocator, input, output))) {
LOG_WARN("invalid input value", K(ret), K(buf_len), K(input));
} else {
const char* src_ptr = src_string.ptr();
int64_t src_pos = 0;
char* dst_ptr = buff;
int64_t dst_pos = 0;
int32_t length = 0;
int32_t wc = 0;
while (OB_SUCC(ret) && src_pos < src_string.length()) {
OZ(ObCharset::mb_wc(src_collation, src_ptr + src_pos, src_string.length() - src_pos, length, wc));
OX(src_pos += length);
int32_t wc_before_process = wc;
if (wc == 0x3000) {
wc = 0x20;
// smart quote not support https://gerry.lamost.org/blog/?p=295757
//} else if (wc == 0x201D) { // " --> " smart double quote
// wc = 0x22;
//} else if (wc == 0x2019) { // ' --> ' smart single quote
// wc = 0x27;
} else if (wc >= 0xFF01 && wc <= 0xFF5E) {
wc -= 65248;
} else {
// do nothing
}
OZ(ObCharset::wc_mb(src_collation, wc, dst_ptr + dst_pos, buff_len - dst_pos, length));
OX(dst_pos += length);
LOG_DEBUG("process char", K(ret), K(src_pos), K(wc_before_process), K(wc));
}
OX(dst_string.assign_ptr(dst_ptr, dst_pos));
pos = output.length();
}
OX(result.set_common_value(dst_string));
OX(result.set_meta_type(result_type_));
} else {
ObStringScanner scanner(input, cs_type);
ObString encoding;
int32_t wc = 0;
char *ptr = buf;
while (OB_SUCC(ret)
&& scanner.next_character(encoding, wc, ret)) {
int32_t length = 0;
if (wc == 0x3000) { //处理空格
wc = 0x20;
//smart quote not support https://gerry.lamost.org/blog/?p=295757
//} else if (wc == 0x201D) { // ” --> " smart double quote
// wc = 0x22;
//} else if (wc == 0x2019) { // ’ --> ' smart single quote
// wc = 0x27;
} else if (wc >= 0xFF01 && wc <= 0xFF5E) {
wc -= 65248;
} else {
//do nothing
}
OZ (ObCharset::wc_mb(cs_type, wc, ptr, buf + buf_len - ptr, length));
ptr += length;
LOG_DEBUG("process char", K(ret), K(wc));
}
pos = ptr - buf;
}
return ret;
}
int ObExprToSingleByte::calc_result1(ObObj &result,
const ObObj &obj,
ObExprCtx &expr_ctx) const
{
int ret = OB_SUCCESS;
ObString src_string;
ObCollationType src_collation = obj.get_collation_type();
if (obj.is_null_oracle()) {
result.set_null();
} else {
CK (OB_NOT_NULL(expr_ctx.calc_buf_));
OZ (obj.get_string(src_string));
if (OB_SUCC(ret)) {
char* buff = NULL;
int64_t buff_len = src_string.length();
int32_t pos = 0;
if (OB_ISNULL(buff = static_cast<char*>(expr_ctx.calc_buf_->alloc(buff_len)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("fail to allocate buffer", K(ret), K(buff_len));
} else if (OB_FAIL(calc_to_single_byte_expr(src_string, src_collation, buff, buff_len, pos))) {
LOG_WARN("fail to calc", K(ret));
} else {
result.set_common_value(ObString(pos, buff));
result.set_meta_type(result_type_);
}
}
}
return ret;
}
int ObExprToSingleByte::cg_expr(ObExprCGCtx &op_cg_ctx,
const ObRawExpr &raw_expr,
ObExpr &rt_expr) const
{
int ret = OB_SUCCESS;
UNUSED(op_cg_ctx);
UNUSED(raw_expr);
rt_expr.eval_func_ = calc_to_single_byte;
return ret;
}
int ObExprToSingleByte::calc_to_single_byte(const ObExpr &expr,
ObEvalCtx &ctx,
ObDatum &res_datum)
{
int ret = OB_SUCCESS;
ObDatum *src_param = NULL;
if (expr.args_[0]->eval(ctx, src_param)) {
LOG_WARN("eval arg failed", K(ret));
} else {
if (src_param->is_null()) {
res_datum.set_null();
} else {
ObString src = src_param->get_string();
char *buf = NULL;
int64_t buf_len = src.length();
int32_t length = 0;
if (OB_ISNULL(buf = static_cast<char*>(expr.get_str_res_mem(ctx, buf_len)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("fail to allocate memory", K(ret), K(src));
} else if (OB_FAIL(calc_to_single_byte_expr(src, expr.args_[0]->datum_meta_.cs_type_,
buf, buf_len, length))) {
LOG_WARN("fail to calc unistr", K(ret));
} else {
res_datum.set_string(buf, length);
}
}
}
return ret;
}
} // namespace sql
} // namespace oceanbase

View File

@ -21,9 +21,14 @@ class ObExprToSingleByte : public ObFuncExprOperator {
public:
explicit ObExprToSingleByte(common::ObIAllocator& alloc);
virtual ~ObExprToSingleByte();
int calc_result_type1(ObExprResType& type, ObExprResType& type1, common::ObExprTypeCtx& type_ctx) const;
int calc_result1(common::ObObj& result, const common::ObObj& obj, common::ObExprCtx& expr_ctx) const;
int calc_result_type1(ObExprResType &type,
ObExprResType &type1,
common::ObExprTypeCtx &type_ctx) const;
int calc_result1(common::ObObj &result,
const common::ObObj &obj,
common::ObExprCtx &expr_ctx) const;
int cg_expr(ObExprCGCtx &op_cg_ctx, const ObRawExpr &raw_expr, ObExpr &rt_expr) const;
static int calc_to_single_byte(const ObExpr &expr, ObEvalCtx &ctx, ObDatum &res_datum);
private:
DISALLOW_COPY_AND_ASSIGN(ObExprToSingleByte);
};