patch 4.0
This commit is contained in:
@ -12,8 +12,12 @@
|
||||
|
||||
#ifndef __CREATE_OP_UTIL_TEST_H__
|
||||
#define __CREATE_OP_UTIL_TEST_H__
|
||||
namespace oceanbase {
|
||||
namespace sql {}
|
||||
} // namespace oceanbase
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace sql
|
||||
{
|
||||
|
||||
}
|
||||
}
|
||||
#endif /* __CREATE_OP_UTIL_TEST_H__ */
|
||||
//// end of header file
|
||||
|
||||
415
unittest/sql/executor/ob_mock_utils.cpp
Normal file
415
unittest/sql/executor/ob_mock_utils.cpp
Normal file
@ -0,0 +1,415 @@
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include "ob_mock_utils.h"
|
||||
#include "sql/executor/ob_distributed_scheduler.h"
|
||||
#include "sql/executor/ob_task_event.h"
|
||||
#include "sql/executor/ob_interm_result_manager.h"
|
||||
#include "sql/executor/ob_local_job_executor.h"
|
||||
#include "sql/executor/ob_local_task_executor.h"
|
||||
#include "sql/executor/ob_distributed_task_runner.h"
|
||||
#include "sql/executor/ob_distributed_transmit.h"
|
||||
#include "sql/engine/ob_physical_plan_ctx.h"
|
||||
#include "rpc/obrpc/ob_rpc_packet.h"
|
||||
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::obrpc;
|
||||
using namespace oceanbase::storage;
|
||||
using namespace oceanbase::share;
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace sql
|
||||
{
|
||||
|
||||
ObMockSqlExecutorRpc::ObMockSqlExecutorRpc()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
for (int64_t i = 1; OB_SUCC(ret) && i <= 5; ++i) {
|
||||
for (int64_t j = 1; OB_SUCC(ret) && j <= 10; ++j) {
|
||||
ObFakePartitionKey key;
|
||||
key.table_id_ = i;
|
||||
key.partition_id_ = j;
|
||||
ObPartitionLocation location;
|
||||
ObReplicaLocation replica_loc;
|
||||
replica_loc.server_.set_ip_addr("127.0.0.1", (int32_t)j);
|
||||
replica_loc.role_ = LEADER;
|
||||
if (OB_SUCCESS != (ret = location.add(replica_loc))) {
|
||||
SQL_ENG_LOG(WARN, "fail to add replica location", K(ret), K(i), K(j));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
partition_service_.set_col_num(TEST_MOCK_COL_NUM);
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < 100; ++i) {
|
||||
ObNewRow row;
|
||||
ObObj objs[TEST_MOCK_COL_NUM];
|
||||
row.count_ = TEST_MOCK_COL_NUM;
|
||||
row.cells_ = objs;
|
||||
for (int64_t j = 0; j < TEST_MOCK_COL_NUM; ++j) {
|
||||
row.cells_[j].set_int(i);
|
||||
}
|
||||
partition_service_.add_row(row);
|
||||
}
|
||||
}
|
||||
|
||||
ObMockSqlExecutorRpc::~ObMockSqlExecutorRpc()
|
||||
{
|
||||
}
|
||||
|
||||
/*
|
||||
* 提交一个异步task执行请求, 在OB_TASK_NOTIFY_FETCH消息的驱动下收取结果数据
|
||||
*/
|
||||
int ObMockSqlExecutorRpc::task_submit(
|
||||
ObExecContext &ctx,
|
||||
ObTask &task,
|
||||
const common::ObAddr &svr)
|
||||
{
|
||||
UNUSED(ctx);
|
||||
UNUSED(task);
|
||||
UNUSED(svr);
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
const static int64_t S_BUF_SIZE = 100000;
|
||||
char s_buf[S_BUF_SIZE];
|
||||
int64_t s_buf_pos = 0;
|
||||
if (OB_SUCCESS != (ret = task.serialize(s_buf, S_BUF_SIZE, s_buf_pos))) {
|
||||
SQL_EXE_LOG(WARN, "fail to serialize task", K(ret));
|
||||
} else {
|
||||
ObExecContext exec_ctx;
|
||||
|
||||
ObMockSqlExecutorRpc rpc;
|
||||
ObArenaAllocator allocator(ObModIds::OB_SQL_EXEC_CONTEXT);
|
||||
ObMockRemoteExecuteStreamHandle resp_handler(allocator);
|
||||
ObTaskExecutorCtx *executor_ctx = exec_ctx.get_task_executor_ctx();
|
||||
executor_ctx->set_task_executor_rpc(&rpc);
|
||||
executor_ctx->set_task_response_handler(resp_handler);
|
||||
executor_ctx->set_server(svr);
|
||||
executor_ctx->set_partition_service(&partition_service_);
|
||||
|
||||
//exec_ctx.set_query_ip_port(0);
|
||||
ObPhysicalPlan *phy_plan = ObPhysicalPlan::alloc();
|
||||
//ObPhyOperatorFactory *phy_op_factory = new ObPhyOperatorFactory();
|
||||
//phy_plan->set_operator_factory(phy_op_factory);
|
||||
|
||||
ObTask new_task;
|
||||
new_task.set_deserialize_param(exec_ctx, *phy_plan);
|
||||
int64_t new_pos = 0;
|
||||
if (OB_SUCCESS != (ret = new_task.deserialize(s_buf, s_buf_pos, new_pos))) {
|
||||
SQL_EXE_LOG(WARN, "fail to deserialize", K(ret));
|
||||
} else {
|
||||
//执行plan
|
||||
ObPhyOperator *root_op = phy_plan->get_main_query();
|
||||
ObDistributedTransmitInput *trans_input = dynamic_cast<ObDistributedTransmitInput *>(ctx.get_phy_op_input(root_op->get_id()));
|
||||
OB_ASSERT(NULL != trans_input);
|
||||
ObSliceID sid = trans_input->get_ob_slice_id();
|
||||
ObDistributedTaskRunner task_runner;
|
||||
if (OB_SUCCESS != (ret = task_runner.execute(exec_ctx, *phy_plan))) {
|
||||
SQL_EXE_LOG(WARN, "fail execute task", K(sid));
|
||||
}
|
||||
//返回task event
|
||||
ObTaskEvent *task_event = new ObTaskEvent();
|
||||
ObTaskLocation task_loc;
|
||||
//ObAddr remote_server;
|
||||
//remote_server.set_ip_addr("127.0.0.1", 9999);
|
||||
task_loc.set_server(task.remote_server_);
|
||||
task_loc.set_ob_task_id(sid.get_ob_task_id());
|
||||
task_event->init(task_loc, ret);
|
||||
if (OB_SUCCESS != (ret = ObMockPacketQueueThread::get_instance()->packet_queue_.push(task_event))) {
|
||||
SQL_EXE_LOG(WARN, "fail to push packet into queue", K(ret));
|
||||
} else {
|
||||
SQL_EXE_LOG(INFO, "succeed to push packet into queue", "task_event", to_cstring(*task_event));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
/*
|
||||
* 发送一个task并阻塞等待,直到对端返回执行状态
|
||||
* 将执行句柄保存在handler中, 随后可以通过handler收取数据
|
||||
* */
|
||||
int ObMockSqlExecutorRpc::task_execute(
|
||||
ObExecContext &ctx,
|
||||
ObTask &task,
|
||||
const common::ObAddr &svr,
|
||||
RemoteExecuteStreamHandle &handler)
|
||||
{
|
||||
UNUSED(task);
|
||||
UNUSED(svr);
|
||||
UNUSED(handler);
|
||||
|
||||
ObMockSqlExecutorRpc rpc;
|
||||
ObArenaAllocator allocator(ObModIds::OB_SQL_EXEC_CONTEXT);
|
||||
ObMockRemoteExecuteStreamHandle resp_handler(allocator);
|
||||
ObTaskExecutorCtx *executor_ctx = ctx.get_task_executor_ctx();
|
||||
executor_ctx->set_task_executor_rpc(&rpc);
|
||||
executor_ctx->set_task_response_handler(resp_handler);
|
||||
executor_ctx->set_server(svr);
|
||||
executor_ctx->set_partition_service(&partition_service_);
|
||||
SQL_EXE_LOG(INFO, "task_execute");
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
/*
|
||||
* 发送杀死一个task的命令并阻塞等待对端返回执行状态
|
||||
* */
|
||||
int ObMockSqlExecutorRpc::task_kill(
|
||||
ObTaskInfo &task,
|
||||
const common::ObAddr &svr)
|
||||
{
|
||||
UNUSED(task);
|
||||
UNUSED(svr);
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
/*
|
||||
* Task在Worker端执行完成,通知Scheduler启动Task读取结果
|
||||
* */
|
||||
int ObMockSqlExecutorRpc::task_complete(
|
||||
ObTaskEvent &task_event,
|
||||
const common::ObAddr &svr)
|
||||
{
|
||||
UNUSED(task_event);
|
||||
UNUSED(svr);
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
/*
|
||||
* 发送一个task的执行结果,不等待返回
|
||||
* */
|
||||
int ObMockSqlExecutorRpc::task_notify_fetch(
|
||||
ObTaskEvent &task_event,
|
||||
const common::ObAddr &svr)
|
||||
{
|
||||
UNUSED(task_event);
|
||||
UNUSED(svr);
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
/*
|
||||
* 获取一个task的中间结果的所有scanner,阻塞等待直到所有的scanner都返回
|
||||
* */
|
||||
int ObMockSqlExecutorRpc::task_fetch_result(
|
||||
const ObSliceID &ob_slice_id,
|
||||
const common::ObAddr &svr,
|
||||
FetchResultStreamHandle &handler)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObScanner &scanner = *handler.get_result();
|
||||
ObObj tmp_objs[TEST_MOCK_COL_NUM];
|
||||
oceanbase::common::ObNewRow tmp_row;
|
||||
tmp_row.count_ = TEST_MOCK_COL_NUM;
|
||||
tmp_row.cells_ = tmp_objs;
|
||||
ObIntermResultIterator iter;
|
||||
ObIntermResultManager *ir_mgr = ObIntermResultManager::get_instance();
|
||||
assert(NULL != ir_mgr);
|
||||
ObTaskLocation task_loc;
|
||||
task_loc.set_server(svr);
|
||||
task_loc.set_ob_task_id(ob_slice_id.get_ob_task_id());
|
||||
ObIntermResultInfo ir_info;
|
||||
ir_info.init(ob_slice_id);
|
||||
if (OB_SUCCESS != (ret = ir_mgr->get_result(ir_info, iter))) {
|
||||
SQL_EXE_LOG(WARN, "fail to get interm result iterator", K(ret),
|
||||
"ob_slice_id", to_cstring(ob_slice_id));
|
||||
} else if (task_location_exist(task_loc)) {
|
||||
//FIXME 暂时做成这样,第二次访问同一个location时返回OB_ITER_END
|
||||
ret = OB_ITER_END;
|
||||
} else {
|
||||
//FIXME 暂时只弄一个scanner,第二次访问同一个location返回OB_ITER_END
|
||||
while (OB_SUCC(ret)) {
|
||||
if (OB_SUCCESS != (ret = iter.get_next_row(tmp_row))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
SQL_EXE_LOG(WARN, "fail to get next row", K(ret));
|
||||
} else {
|
||||
SQL_EXE_LOG(DEBUG, "iter has no more rows");
|
||||
}
|
||||
} else if (OB_SUCCESS != (ret = scanner.add_row(tmp_row))) {
|
||||
SQL_EXE_LOG(WARN, "fail to add row to scanner", K(ret));
|
||||
} else {
|
||||
SQL_EXE_LOG(DEBUG, "success to add row to scanner");
|
||||
}
|
||||
}
|
||||
if (OB_ITER_END == ret) {
|
||||
if (OB_SUCCESS != (ret = task_loc_array_.push_back(task_loc))) {
|
||||
SQL_EXE_LOG(WARN, "fail to push back to task location array", K(ret));
|
||||
} else{
|
||||
ret = OB_SUCCESS;
|
||||
|
||||
SQL_EXE_LOG(DEBUG, "get a scanner",
|
||||
"job_id", task_loc.get_job_id(),
|
||||
"task_id", task_loc.get_task_id(),
|
||||
"row_count", scanner.get_row_count());
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
SQL_EXE_LOG(INFO, "get interm result", K(ret), K(svr), K(ob_slice_id));
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObMockSqlExecutorRpc::task_location_exist(ObTaskLocation task_loc)
|
||||
{
|
||||
bool loc_exist = false;
|
||||
for (int64_t i = 0; i < task_loc_array_.count(); ++i) {
|
||||
if (task_loc_array_.at(i).get_job_id() == task_loc.get_job_id()
|
||||
&& task_loc_array_.at(i).get_task_id() == task_loc.get_task_id()) {
|
||||
loc_exist = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
return loc_exist;
|
||||
}
|
||||
|
||||
bool ObMockFetchResultStreamHandle::has_more()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
int ObMockRemoteExecuteStreamHandle::get_more(ObScanner &scanner)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
static const int64_t col_num = TEST_MOCK_COL_NUM;
|
||||
for (int64_t i = 0; i < 1000; ++i) {
|
||||
ObNewRow row;
|
||||
ObObj objs[col_num];
|
||||
row.count_ = col_num;
|
||||
row.cells_ = objs;
|
||||
for (int64_t j = 0; j < col_num; ++j) {
|
||||
row.cells_[j].set_int(i);
|
||||
}
|
||||
if (OB_SUCCESS != (ret = scanner.add_row(row))) {
|
||||
SQL_EXE_LOG(ERROR, "fail to add row", K(i), K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
||||
bool ObMockRemoteExecuteStreamHandle::has_more()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
int ObMockFetchResultStreamHandle::get_more(ObScanner &scanner)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObObj tmp_objs[TEST_MOCK_COL_NUM];
|
||||
oceanbase::common::ObNewRow tmp_row;
|
||||
tmp_row.count_ = TEST_MOCK_COL_NUM;
|
||||
tmp_row.cells_ = tmp_objs;
|
||||
ObIntermResultIterator iter;
|
||||
ObIntermResultManager *ir_mgr = ObIntermResultManager::get_instance();
|
||||
assert(NULL != ir_mgr);
|
||||
ObTaskLocation task_loc;
|
||||
task_loc.set_server(server_);
|
||||
task_loc.set_ob_task_id(ob_slice_id_.get_ob_task_id());
|
||||
ObIntermResultInfo ir_info;
|
||||
ir_info.init(ob_slice_id_);
|
||||
if (OB_SUCCESS != (ret = ir_mgr->get_result(ir_info, iter))) {
|
||||
SQL_EXE_LOG(WARN, "fail to get interm result iterator", K(ret),
|
||||
"ob_slice_id", to_cstring(ob_slice_id_));
|
||||
} else if (task_location_exist(task_loc)) {
|
||||
//FIXME 暂时做成这样,第二次访问同一个location时返回OB_ITER_END
|
||||
ret = OB_ITER_END;
|
||||
} else {
|
||||
//FIXME 暂时只弄一个scanner,第二次访问同一个location返回OB_ITER_END
|
||||
while (OB_SUCC(ret)) {
|
||||
if (OB_SUCCESS != (ret = iter.get_next_row(tmp_row))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
SQL_EXE_LOG(WARN, "fail to get next row", K(ret));
|
||||
} else {
|
||||
SQL_EXE_LOG(DEBUG, "iter has no more rows");
|
||||
}
|
||||
} else if (OB_SUCCESS != (ret = scanner.add_row(tmp_row))) {
|
||||
SQL_EXE_LOG(WARN, "fail to add row to scanner", K(ret));
|
||||
} else {
|
||||
SQL_EXE_LOG(DEBUG, "success to add row to scanner");
|
||||
}
|
||||
}
|
||||
if (OB_ITER_END == ret) {
|
||||
if (OB_SUCCESS != (ret = task_loc_array_.push_back(task_loc))) {
|
||||
SQL_EXE_LOG(WARN, "fail to push back to task location array", K(ret));
|
||||
} else{
|
||||
ret = OB_SUCCESS;
|
||||
|
||||
SQL_EXE_LOG(DEBUG, "get a scanner",
|
||||
"job_id", task_loc.get_job_id(),
|
||||
"task_id", task_loc.get_task_id(),
|
||||
"row_count", scanner.get_row_count());
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
SQL_EXE_LOG(INFO, "get interm result", K(ret), K_(server), K_(ob_slice_id));
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
||||
bool ObMockFetchResultStreamHandle::task_location_exist(ObTaskLocation task_loc)
|
||||
{
|
||||
bool loc_exist = false;
|
||||
for (int64_t i = 0; i < task_loc_array_.count(); ++i) {
|
||||
if (task_loc_array_.at(i).get_job_id() == task_loc.get_job_id()
|
||||
&& task_loc_array_.at(i).get_task_id() == task_loc.get_task_id()) {
|
||||
loc_exist = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
return loc_exist;
|
||||
}
|
||||
|
||||
ObMockPacketQueueThread *ObMockPacketQueueThread::instance_ = NULL;
|
||||
obutil::Mutex ObMockPacketQueueThread::locker_;
|
||||
|
||||
ObMockPacketQueueThread::ObMockPacketQueueThread()
|
||||
{
|
||||
packet_queue_.init(8192, common::ObModIds::OB_SQL_EXECUTOR_TASK_EVENT);
|
||||
set_thread_count(THREAD_COUNT);
|
||||
}
|
||||
|
||||
ObMockPacketQueueThread *ObMockPacketQueueThread::get_instance()
|
||||
{
|
||||
if (NULL == instance_) {
|
||||
locker_.lock();
|
||||
if (NULL == instance_) {
|
||||
instance_ = new ObMockPacketQueueThread();
|
||||
if (NULL == instance_) {
|
||||
SQL_EXE_LOG(ERROR, "instance is NULL, unexpected");
|
||||
} else {
|
||||
//empty
|
||||
}
|
||||
}
|
||||
locker_.unlock();
|
||||
}
|
||||
return instance_;
|
||||
}
|
||||
|
||||
void ObMockPacketQueueThread::run(obsys::CThread *thread, void *arg)
|
||||
{
|
||||
SQL_EXE_LOG(INFO, "mock packet queue is running...");
|
||||
|
||||
UNUSED(arg);
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t timeout_us = INT64_MAX;
|
||||
void *msg = NULL;
|
||||
ObDistributedSchedulerManager *sc_manager = ObDistributedSchedulerManager::get_instance();
|
||||
while(OB_SUCCESS == (ret = ObMockPacketQueueThread::get_instance()->packet_queue_.pop(timeout_us, msg))) {
|
||||
ObTaskEvent *packet = reinterpret_cast<ObTaskEvent*>(msg);
|
||||
if (NULL == packet) {
|
||||
SQL_EXE_LOG(ERROR, "packet is NULL");
|
||||
} else if (OB_SUCCESS != (ret = sc_manager->signal_scheduler(*packet))) {
|
||||
SQL_EXE_LOG(WARN, "fail to signal scheduler", K(ret));
|
||||
} else {
|
||||
//empty
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}/* ns sql*/
|
||||
}/* ns oceanbase */
|
||||
@ -15,97 +15,125 @@
|
||||
|
||||
#include "sql/executor/ob_executor_rpc_impl.h"
|
||||
#include "lib/queue/ob_spop_mpush_queue.h"
|
||||
#include "../engine/table/ob_fake_partition_location_cache.h"
|
||||
#include "../engine/table/ob_fake_partition_service.h"
|
||||
#include "create_op_util.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace sql
|
||||
{
|
||||
static const int64_t TEST_MOCK_COL_NUM = 3;
|
||||
|
||||
class ObMockSqlExecutorRpc : public ObExecutorRpcImpl {
|
||||
class ObMockSqlExecutorRpc : public ObExecutorRpcImpl
|
||||
{
|
||||
public:
|
||||
ObMockSqlExecutorRpc();
|
||||
virtual ~ObMockSqlExecutorRpc();
|
||||
virtual int task_submit(ObExecContext& ctx, ObTask& task, const common::ObAddr& svr);
|
||||
/*
|
||||
* 提交一个异步task执行请求, 在OB_TASK_NOTIFY_FETCH消息的驱动下收取结果数据
|
||||
*/
|
||||
virtual int task_submit(
|
||||
ObExecContext &ctx,
|
||||
ObTask &task,
|
||||
const common::ObAddr &svr);
|
||||
/*
|
||||
* 发送一个task并阻塞等待,直到对端返回执行状态
|
||||
* 将执行句柄保存在handler中, 随后可以通过handler收取数据
|
||||
* */
|
||||
virtual int task_execute(
|
||||
ObExecContext& ctx, ObTask& task, const common::ObAddr& svr, RemoteExecuteStreamHandle& handler);
|
||||
virtual int task_kill(ObTaskInfo& task, const common::ObAddr& svr);
|
||||
virtual int task_complete(ObTaskEvent& task_event, const common::ObAddr& svr);
|
||||
virtual int task_notify_fetch(ObTaskEvent& task_event, const common::ObAddr& svr);
|
||||
ObExecContext &ctx,
|
||||
ObTask &task,
|
||||
const common::ObAddr &svr,
|
||||
RemoteExecuteStreamHandle &handler);
|
||||
/*
|
||||
* 发送杀死一个task的命令并阻塞等待对端返回执行状态
|
||||
* */
|
||||
virtual int task_kill(
|
||||
ObTaskInfo &task,
|
||||
const common::ObAddr &svr);
|
||||
/*
|
||||
* Task在Worker端执行完成,通知Scheduler启动Task读取结果
|
||||
* */
|
||||
virtual int task_complete(
|
||||
ObTaskEvent &task_event,
|
||||
const common::ObAddr &svr);
|
||||
|
||||
/*
|
||||
* 发送一个task的执行结果,不等待返回
|
||||
* */
|
||||
virtual int task_notify_fetch(
|
||||
ObTaskEvent &task_event,
|
||||
const common::ObAddr &svr);
|
||||
/*
|
||||
* 获取一个task的中间结果的所有scanner,阻塞等待直到所有的scanner都返回
|
||||
* */
|
||||
virtual int task_fetch_result(
|
||||
const ObSliceID& ob_slice_id, const common::ObAddr& svr, FetchResultStreamHandle& handler);
|
||||
const ObSliceID &ob_slice_id,
|
||||
const common::ObAddr &svr,
|
||||
FetchResultStreamHandle &handler);
|
||||
|
||||
public:
|
||||
share::ObFakePartitionLocationCache partition_loc_cache_;
|
||||
storage::ObFakePartitionService partition_service_;
|
||||
|
||||
private:
|
||||
bool task_location_exist(ObTaskLocation task_loc);
|
||||
|
||||
private:
|
||||
common::ObArray<ObTaskLocation> task_loc_array_;
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObMockSqlExecutorRpc);
|
||||
};
|
||||
|
||||
class ObMockRemoteExecuteStreamHandle : public RemoteExecuteStreamHandle {
|
||||
class ObMockRemoteExecuteStreamHandle : public RemoteExecuteStreamHandle
|
||||
{
|
||||
public:
|
||||
ObMockRemoteExecuteStreamHandle(common::ObIAllocator& alloc) : RemoteExecuteStreamHandle(alloc)
|
||||
ObMockRemoteExecuteStreamHandle(common::ObIAllocator &alloc) : RemoteExecuteStreamHandle(alloc)
|
||||
{}
|
||||
~ObMockRemoteExecuteStreamHandle()
|
||||
{}
|
||||
virtual int get_more(ObScanner& scanner);
|
||||
virtual int get_more(ObScanner &scanner);
|
||||
virtual bool has_more();
|
||||
};
|
||||
|
||||
class ObMockFetchResultStreamHandle : public FetchResultStreamHandle {
|
||||
|
||||
class ObMockFetchResultStreamHandle : public FetchResultStreamHandle
|
||||
{
|
||||
public:
|
||||
ObMockFetchResultStreamHandle(common::ObIAllocator& alloc) : FetchResultStreamHandle(alloc)
|
||||
ObMockFetchResultStreamHandle(common::ObIAllocator &alloc) : FetchResultStreamHandle(alloc)
|
||||
{}
|
||||
~ObMockFetchResultStreamHandle()
|
||||
{}
|
||||
virtual int get_more(ObScanner& scanner);
|
||||
virtual int get_more(ObScanner &scanner);
|
||||
virtual bool has_more();
|
||||
|
||||
void set_server(const common::ObAddr& server)
|
||||
{
|
||||
server_ = server;
|
||||
}
|
||||
void set_slice_id(const ObSliceID& ob_slice_id)
|
||||
{
|
||||
ob_slice_id_ = ob_slice_id;
|
||||
}
|
||||
|
||||
void set_server(const common::ObAddr &server) { server_ = server; }
|
||||
void set_slice_id(const ObSliceID &ob_slice_id) { ob_slice_id_ = ob_slice_id; }
|
||||
private:
|
||||
bool task_location_exist(ObTaskLocation task_loc);
|
||||
|
||||
private:
|
||||
// task_submit的时候填入进来
|
||||
common::ObArray<ObTaskLocation> task_loc_array_;
|
||||
common::ObAddr server_;
|
||||
ObSliceID ob_slice_id_;
|
||||
};
|
||||
|
||||
/************************************mock packet queue********************************/
|
||||
class ObMockPacketQueueThread : public share::ObThreadPool {
|
||||
/************************************模拟 packet queue********************************/
|
||||
class ObMockPacketQueueThread : public share::ObThreadPool
|
||||
{
|
||||
public:
|
||||
static const int64_t THREAD_COUNT = 1;
|
||||
static ObMockPacketQueueThread* get_instance();
|
||||
static ObMockPacketQueueThread *get_instance();
|
||||
|
||||
ObMockPacketQueueThread();
|
||||
virtual ~ObMockPacketQueueThread()
|
||||
{}
|
||||
virtual ~ObMockPacketQueueThread() {}
|
||||
|
||||
void run1();
|
||||
|
||||
common::ObSPopMPushQueue packet_queue_;
|
||||
|
||||
private:
|
||||
static ObMockPacketQueueThread* instance_;
|
||||
static ObMockPacketQueueThread *instance_;
|
||||
static obutil::Mutex locker_;
|
||||
};
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
}
|
||||
}
|
||||
#endif /* OCEANBASE_SQL_EXECUTOR_OB_MOCK_SQL_EXECUTOR_RPC_ */
|
||||
|
||||
384
unittest/sql/executor/rpc_remote_scheduler.cpp
Normal file
384
unittest/sql/executor/rpc_remote_scheduler.cpp
Normal file
@ -0,0 +1,384 @@
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
#include <stdarg.h>
|
||||
#include "lib/utility/ob_tracepoint.h"
|
||||
#include "sql/executor/ob_remote_scheduler.h"
|
||||
//#include "ob_mock_utils.h"
|
||||
#include "sql/engine/table/ob_table_scan.h"
|
||||
#include "sql/executor/ob_root_transmit.h"
|
||||
#include "sql/executor/ob_direct_transmit.h"
|
||||
#include "sql/executor/ob_direct_receive.h"
|
||||
#include "sql/executor/ob_remote_job_control.h"
|
||||
#include "sql/executor/ob_job_parser.h"
|
||||
#include "sql/executor/ob_task_spliter_factory.h"
|
||||
#include "sql/executor/ob_executor_rpc_impl.h"
|
||||
#include "sql/engine/ob_physical_plan_ctx.h"
|
||||
#include "sql/plan_cache/ob_cache_object_factory.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/ob_physical_plan_ctx.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
#include "sql/ob_sql_init.h"
|
||||
#include "sql/optimizer/ob_table_location.h"
|
||||
#include "sql/executor/ob_executor_rpc_proxy.h"
|
||||
#include "rpc/obrpc/ob_net_client.h"
|
||||
#include "storage/tx/ob_trans_define.h"
|
||||
|
||||
using namespace oceanbase;
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
using namespace oceanbase::obrpc;
|
||||
using namespace oceanbase::share;
|
||||
using namespace oceanbase::transaction;
|
||||
|
||||
class ObRemoteSchedulerTest : public ::testing::Test
|
||||
{
|
||||
public:
|
||||
const static int64_t TEST_MOCK_COL_NUM = 3;
|
||||
|
||||
const static int64_t TEST_SPLIT_TASK_COUNT = 7;
|
||||
const static int64_t TEST_PARA_SERVER_COUNT = 2;
|
||||
const static int64_t TEST_SERVER_PARA_THREAD_COUNT = 3;
|
||||
|
||||
const static int64_t TEST_TABLE_ID = 1099511627778;
|
||||
const static int64_t TEST_INDEX_ID = 1099511627778;
|
||||
//const static int64_t TEST_TABLE_ID = 1099511627777;//1099511627778;
|
||||
//const static int64_t TEST_INDEX_ID = 1099511627777;//1099511627778;
|
||||
const static uint64_t COLUMN_ID_1 = 16;
|
||||
const static uint64_t COLUMN_ID_2 = 17;
|
||||
|
||||
const static int64_t TEST_LIMIT = 10;
|
||||
const static int64_t TEST_OFFSET = 0;
|
||||
|
||||
ObRemoteSchedulerTest();
|
||||
virtual ~ObRemoteSchedulerTest();
|
||||
virtual void SetUp();
|
||||
virtual void TearDown();
|
||||
|
||||
ObPhysicalPlan *phy_plan_;
|
||||
|
||||
int create_plan_tree(ObExecContext &ctx);
|
||||
|
||||
private:
|
||||
// disallow copy
|
||||
ObRemoteSchedulerTest(const ObRemoteSchedulerTest &other);
|
||||
ObRemoteSchedulerTest& operator=(const ObRemoteSchedulerTest &other);
|
||||
private:
|
||||
// data members
|
||||
};
|
||||
ObRemoteSchedulerTest::ObRemoteSchedulerTest()
|
||||
{
|
||||
}
|
||||
|
||||
ObRemoteSchedulerTest::~ObRemoteSchedulerTest()
|
||||
{
|
||||
}
|
||||
|
||||
void ObRemoteSchedulerTest::SetUp()
|
||||
{
|
||||
}
|
||||
|
||||
void ObRemoteSchedulerTest::TearDown()
|
||||
{
|
||||
}
|
||||
|
||||
int ObRemoteSchedulerTest::create_plan_tree(ObExecContext &ctx)
|
||||
{
|
||||
UNUSED(ctx);
|
||||
int ret = OB_SUCCESS;
|
||||
ObCacheObjectFactory::alloc(phy_plan_);
|
||||
//phy_plan_ = ObPhysicalPlan::alloc();
|
||||
ObPhyOperator *cur_op = NULL;
|
||||
ObPhyOperator *tmp_op = NULL;
|
||||
|
||||
ObSqlExpression *limit_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(phy_plan_, limit_expr));
|
||||
EXPECT_FALSE(NULL == limit_expr);
|
||||
ObPostExprItem limit_expr_item;
|
||||
limit_expr_item.set_int(TEST_LIMIT);
|
||||
limit_expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, limit_expr->add_expr_item(limit_expr_item));
|
||||
|
||||
ObSqlExpression *offset_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(phy_plan_, offset_expr));
|
||||
EXPECT_FALSE(NULL == offset_expr);
|
||||
ObPostExprItem offset_expr_item;
|
||||
offset_expr_item.set_int(TEST_OFFSET);
|
||||
offset_expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, offset_expr->add_expr_item(offset_expr_item));
|
||||
|
||||
/*
|
||||
* calculate c0 % TEST_SPLIT_TASK_COUNT
|
||||
* */
|
||||
ObSqlExpression *hash_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(phy_plan_, hash_expr));
|
||||
EXPECT_FALSE(NULL == hash_expr);
|
||||
ObPostExprItem expr_item;
|
||||
expr_item.set_column(0);
|
||||
EXPECT_EQ(OB_SUCCESS, hash_expr->add_expr_item(expr_item));
|
||||
expr_item.set_int(TEST_SPLIT_TASK_COUNT);
|
||||
expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, hash_expr->add_expr_item(expr_item));
|
||||
expr_item.set_op(phy_plan_->get_allocator(), "%", 2);
|
||||
EXPECT_EQ(OB_SUCCESS, hash_expr->add_expr_item(expr_item));
|
||||
|
||||
/*
|
||||
* calculate c0 % 1
|
||||
* */
|
||||
ObSqlExpression *iden_hash_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(phy_plan_, iden_hash_expr));
|
||||
EXPECT_FALSE(NULL == iden_hash_expr);
|
||||
ObPostExprItem iden_expr_item;
|
||||
iden_expr_item.set_column(0);
|
||||
EXPECT_EQ(OB_SUCCESS, iden_hash_expr->add_expr_item(iden_expr_item));
|
||||
iden_expr_item.set_int(1);
|
||||
iden_expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, iden_hash_expr->add_expr_item(iden_expr_item));
|
||||
iden_expr_item.set_op(phy_plan_->get_allocator(), "%", 2);
|
||||
EXPECT_EQ(OB_SUCCESS, iden_hash_expr->add_expr_item(iden_expr_item));
|
||||
|
||||
|
||||
ObPhysicalPlanCtx *plan_ctx = ctx.get_physical_plan_ctx();
|
||||
common::ObIArray<common::ObObjParam> ¶m_store = plan_ctx->get_param_store_for_update();
|
||||
ObObjParam value1;
|
||||
value1.set_int(3);
|
||||
param_store.push_back(value1);
|
||||
ObObjParam value2;
|
||||
value2.set_int(1);
|
||||
param_store.push_back(value2);
|
||||
ObObjParam value3;
|
||||
value3.set_int(2);
|
||||
param_store.push_back(value3);
|
||||
|
||||
|
||||
/*
|
||||
* calculate a + ?
|
||||
* */
|
||||
ObSqlExpression *partition_func = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(phy_plan_, partition_func));
|
||||
EXPECT_FALSE(NULL == partition_func);
|
||||
ObPostExprItem partition_func_item;
|
||||
partition_func_item.set_column(0);
|
||||
EXPECT_EQ(OB_SUCCESS, partition_func->add_expr_item(partition_func_item));
|
||||
ObObj index_value;
|
||||
index_value.set_int(2);
|
||||
partition_func_item.assign(index_value);
|
||||
EXPECT_EQ(OB_SUCCESS, partition_func->add_expr_item(partition_func_item));
|
||||
partition_func_item.set_op(phy_plan_->get_allocator(), "+", 2);
|
||||
EXPECT_EQ(OB_SUCCESS, partition_func->add_expr_item(partition_func_item));
|
||||
|
||||
ObColumnRefRawExpr ref_col1(TEST_TABLE_ID, COLUMN_ID_1, T_REF_COLUMN);
|
||||
//ObArray<ColumnItem> single_range_columns;
|
||||
ColumnItem col1;
|
||||
ref_col1.set_data_type(ObIntType);
|
||||
ref_col1.set_column_attr(ObString::make_string(""), ObString::make_string("a"));
|
||||
col1.table_id_ = ref_col1.get_table_id();
|
||||
col1.column_id_ = ref_col1.get_column_id();
|
||||
col1.column_name_ = ref_col1.get_column_name();
|
||||
col1.expr_ = &ref_col1;
|
||||
ref_col1.add_flag(IS_COLUMN);
|
||||
ObColumnRefRawExpr ref_col2(TEST_TABLE_ID, COLUMN_ID_2, T_REF_COLUMN);
|
||||
ColumnItem col2;
|
||||
ref_col2.set_data_type(ObIntType);
|
||||
ref_col1.set_column_attr(ObString::make_string(""), ObString::make_string("b"));
|
||||
col1.table_id_ = ref_col2.get_table_id();
|
||||
col1.column_id_ = ref_col2.get_column_id();
|
||||
col2.column_name_ = ref_col2.get_column_name();
|
||||
col2.expr_ = &ref_col2;
|
||||
ref_col2.add_flag(IS_COLUMN);
|
||||
|
||||
// 构造 (a = ?)
|
||||
ObObj index1;
|
||||
index1.set_unknown(0);
|
||||
ObConstRawExpr const_col1(index1, T_QUESTIONMARK);
|
||||
const_col1.add_flag(IS_STATIC_PARAM);
|
||||
ObOpRawExpr condition1(&ref_col1, &const_col1, T_OP_EQ); // a = ?构造完毕
|
||||
// 构造 (b > ?)
|
||||
ObObj index2;
|
||||
index2.set_unknown(1);
|
||||
ObConstRawExpr const_col2(index2, T_QUESTIONMARK);
|
||||
const_col2.add_flag(IS_STATIC_PARAM);
|
||||
ObOpRawExpr condition2(&ref_col2, &const_col2, T_OP_GT); // b > ?构造完毕
|
||||
ObOpRawExpr condition3(&condition1, &condition2, T_OP_AND); // a = ? and b > ?构造完毕
|
||||
|
||||
ObArray<ColumnItem> scan_range_columns;
|
||||
EXPECT_EQ(OB_SUCCESS, scan_range_columns.push_back(col2));
|
||||
ObArray<ColumnItem> partition_range_columns;
|
||||
EXPECT_EQ(OB_SUCCESS, partition_range_columns.push_back(col1));
|
||||
ObQueryRange *query_range = OB_NEW(ObQueryRange, ObModIds::TEST);
|
||||
EXPECT_EQ(OB_SUCCESS, query_range->preliminary_extract_query_range(scan_range_columns, &condition3, NULL));
|
||||
EXPECT_EQ(OB_SUCCESS, query_range->preliminary_extract_query_range(partition_range_columns, &condition3, NULL));
|
||||
|
||||
EXPECT_EQ(OB_SUCCESS, phy_plan_->alloc_operator_by_type(PHY_TABLE_SCAN, tmp_op));
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_table_location_key(TEST_TABLE_ID);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_ref_table_id(TEST_TABLE_ID);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_index_table_id(TEST_INDEX_ID);
|
||||
static_cast<ObTableScan*>(tmp_op)->add_output_column(COLUMN_ID_1);
|
||||
static_cast<ObTableScan*>(tmp_op)->add_output_column(COLUMN_ID_2);
|
||||
//static_cast<ObTableScan*>(tmp_op)->set_limit_offset(*limit_expr, *offset_expr);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_query_range(*query_range);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
EXPECT_EQ(OB_SUCCESS, phy_plan_->alloc_operator_by_type(PHY_DIRECT_TRANSMIT, tmp_op));
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::REMOTE_IDENTITY_SPLIT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_split_task_count(TEST_SPLIT_TASK_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_parallel_server_count(TEST_PARA_SERVER_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_server_parallel_thread_count(TEST_SERVER_PARA_THREAD_COUNT);
|
||||
//static_cast<ObTransmit*>(tmp_op)->set_shuffle_func(hash_expr);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
EXPECT_EQ(OB_SUCCESS, phy_plan_->alloc_operator_by_type(PHY_DIRECT_RECEIVE, tmp_op));
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
EXPECT_EQ(OB_SUCCESS, phy_plan_->alloc_operator_by_type(PHY_ROOT_TRANSMIT, tmp_op));
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::LOCAL_IDENTITY_SPLIT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_split_task_count(TEST_SPLIT_TASK_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_parallel_server_count(TEST_PARA_SERVER_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_server_parallel_thread_count(TEST_SERVER_PARA_THREAD_COUNT);
|
||||
//static_cast<ObTransmit*>(tmp_op)->set_shuffle_func(hash_expr);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
phy_plan_->set_main_query(cur_op);
|
||||
|
||||
//EXPECT_EQ(OB_SUCCESS, phy_plan_->add_table_id(TEST_TABLE_ID));
|
||||
phy_plan_->set_location_type(OB_PHY_PLAN_REMOTE);
|
||||
|
||||
SQL_EXE_LOG(INFO, "physical plan", K(*phy_plan_));
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
TEST_F(ObRemoteSchedulerTest, basic_test)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
//启动模拟收包队列
|
||||
//ObMockPacketQueueThread::get_instance()->start();
|
||||
|
||||
ObRemoteScheduler remote_scheduler;
|
||||
|
||||
ObArenaAllocator allocator(ObModIds::TEST);
|
||||
ObAddr dst_server;
|
||||
dst_server.set_ip_addr("10.125.224.8", 38455);
|
||||
//dst_server.set_ip_addr("10.125.224.9", 60593);
|
||||
ObNetClient client;
|
||||
ObExecutorRpcProxy proxy;
|
||||
ASSERT_EQ(OB_SUCCESS, client.init());
|
||||
ASSERT_EQ(OB_SUCCESS, client.get_proxy(proxy));
|
||||
const int64_t timeout = 180 * 1000 * 1000; //180s
|
||||
proxy.set_timeout(timeout);
|
||||
proxy.set_server(dst_server);
|
||||
ObExecutorRpcImpl rpc;
|
||||
rpc.set_rpc_proxy(&proxy);
|
||||
|
||||
ObPhyTableLocationFixedArray table_locs;
|
||||
ObPhyTableLocation table_loc;
|
||||
ObPartitionReplicaLocation partition_loc;
|
||||
ObReplicaLocation replica_loc;
|
||||
replica_loc.server_ = dst_server;
|
||||
replica_loc.role_ = LEADER;
|
||||
partition_loc.set_table_id(TEST_TABLE_ID);
|
||||
partition_loc.set_partition_id(0);
|
||||
partition_loc.set_replica_location(replica_loc);
|
||||
table_loc.set_table_location_key(TEST_TABLE_ID, TEST_TABLE_ID);
|
||||
ASSERT_EQ(OB_SUCCESS, table_loc.add_partition_location(partition_loc));
|
||||
ASSERT_EQ(OB_SUCCESS, table_locs.push_back(table_loc));
|
||||
|
||||
ObAddr server;
|
||||
server.set_ip_addr("10.125.224.8", 38455);
|
||||
ObTransID ob_trans_id;
|
||||
ObTransDesc trans_desc;
|
||||
trans_desc.set_trans_id(ob_trans_id);
|
||||
trans_desc.set_snapshot_version(0);
|
||||
//RemoteExecuteStreamHandle resp_handler(allocator);
|
||||
ObExecuteResult exe_result;
|
||||
ObExecContext exec_ctx;
|
||||
exec_ctx.init_phy_op(100);
|
||||
exec_ctx.create_physical_plan_ctx();
|
||||
ObPhysicalPlanCtx *plan_ctx = exec_ctx.get_physical_plan_ctx();
|
||||
ObSQLSessionInfo *my_session = exec_ctx.get_my_session();
|
||||
//plan_ctx->set_server(server);
|
||||
plan_ctx->set_timeout_timestamp(::oceanbase::common::ObTimeUtility::current_time() + 2000L * 1000L);
|
||||
ASSERT_EQ(OB_SUCCESS, my_session->init_tenant(ObString::make_string("t1"), 2));
|
||||
ObObj autocommit_obj, min_val, max_val;
|
||||
ObObj autocommit_type;
|
||||
autocommit_obj.set_varchar("1");
|
||||
min_val.set_varchar("");
|
||||
max_val.set_varchar("");
|
||||
autocommit_type.set_type(ObIntType);
|
||||
ObArenaAllocator calc_buf(ObModIds::OB_SQL_SESSION);
|
||||
ASSERT_EQ(OB_SUCCESS, my_session->load_sys_variable(calc_buf, ObString::make_string(OB_SV_AUTOCOMMIT), autocommit_type, autocommit_obj, min_val, max_val,
|
||||
ObSysVarFlag::GLOBAL_SCOPE | ObSysVarFlag::SESSION_SCOPE | ObSysVarFlag::NEED_SERIALIZE));
|
||||
//my_session->set_autocommit(true);
|
||||
//plan_ctx->get_trans_desc() = trans_desc;
|
||||
ObTaskExecutorCtx *executor_ctx = exec_ctx.get_task_executor_ctx();
|
||||
executor_ctx->set_task_executor_rpc(rpc);
|
||||
//executor_ctx->set_task_response_handler(resp_handler);
|
||||
//executor_ctx->set_partition_location_cache(&rpc.partition_loc_cache_);
|
||||
//executor_ctx->set_partition_service(&rpc.partition_service_);
|
||||
executor_ctx->set_execute_result(&exe_result);
|
||||
executor_ctx->set_table_locations(table_locs);
|
||||
executor_ctx->set_self_addr(server);
|
||||
|
||||
create_plan_tree(exec_ctx);
|
||||
|
||||
int64_t sent_task_count = 0;
|
||||
ASSERT_EQ(OB_SUCCESS, remote_scheduler.schedule(exec_ctx, phy_plan_, sent_task_count));
|
||||
//ObExecutor ob_exe;
|
||||
//ASSERT_EQ(OB_SUCCESS, ob_exe.execute_plan(exec_ctx, phy_plan_));
|
||||
ASSERT_EQ(OB_SUCCESS, exe_result.open(exec_ctx));
|
||||
const ObNewRow *tmp_row = NULL;
|
||||
while(OB_SUCCESS == (ret = exe_result.get_next_row(exec_ctx, tmp_row))) {
|
||||
SQL_EXE_LOG(INFO, "get a row", K(*tmp_row));
|
||||
}
|
||||
ASSERT_EQ(OB_ITER_END, ret);
|
||||
ASSERT_EQ(OB_SUCCESS, exe_result.close(exec_ctx));
|
||||
|
||||
|
||||
#if 0
|
||||
ObArenaAllocator allocator(ObModIds::TEST);
|
||||
ObAddr dst_server;
|
||||
dst_server.set_ip_addr("10.125.224.8", 60593);
|
||||
ObNetClient client;
|
||||
ObExecutorRpcProxy proxy;
|
||||
ASSERT_EQ(OB_SUCCESS, client.init());
|
||||
ASSERT_EQ(OB_SUCCESS, client.get_proxy(proxy));
|
||||
const int64_t timeout = 180 * 1000 * 1000; //180s
|
||||
proxy.set_timeout(timeout);
|
||||
proxy.set_server(dst_server);
|
||||
ObTask task;
|
||||
RemoteExecuteStreamHandle handler(allocator);
|
||||
RemoteExecuteStreamHandle::MyHandle &h = handler.get_handle();
|
||||
ASSERT_EQ(OB_SUCCESS, proxy.task_execute(task, *handler.get_result(), h));
|
||||
#endif
|
||||
}
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
init_sql_factories();
|
||||
oceanbase::common::ObLogger::get_logger().set_log_level("INFO");
|
||||
::testing::InitGoogleTest(&argc,argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
||||
379
unittest/sql/executor/stress_interm_result_manager.cpp
Normal file
379
unittest/sql/executor/stress_interm_result_manager.cpp
Normal file
@ -0,0 +1,379 @@
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
#include "share/ob_thread_pool.h"
|
||||
#include "sql/executor/ob_interm_result_manager.h"
|
||||
#include "sql/executor/ob_interm_result_pool.h"
|
||||
#include "sql/ob_sql_init.h"
|
||||
#include <stdlib.h>
|
||||
#include <math.h>
|
||||
#include <time.h>
|
||||
|
||||
using namespace oceanbase;
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
|
||||
class ThreadContext
|
||||
{
|
||||
public:
|
||||
static const int64_t COL_NUM = 16;
|
||||
|
||||
ThreadContext();
|
||||
virtual ~ThreadContext();
|
||||
|
||||
int add_result(ObIntermResultInfo &ir_info);
|
||||
int read_result(ObIntermResultInfo &ir_info);
|
||||
int read_and_delete_result(ObIntermResultInfo &ir_info);
|
||||
private:
|
||||
int alloc_result(ObIntermResult *&ir);
|
||||
int free_result(ObIntermResult *ir);
|
||||
ObIntermResultManager* ir_manager_;
|
||||
oceanbase::common::ObNewRow row_;
|
||||
oceanbase::common::ObNewRow tmp_row_;
|
||||
ObObj objs_[COL_NUM];
|
||||
ObObj tmp_objs_[COL_NUM];
|
||||
};
|
||||
|
||||
ThreadContext::ThreadContext()
|
||||
{
|
||||
ir_manager_ = ObIntermResultManager::get_instance();
|
||||
|
||||
row_.count_ = COL_NUM;
|
||||
row_.cells_ = objs_;
|
||||
for (int64_t i = 0; i < COL_NUM; ++i) {
|
||||
row_.cells_[i].set_int((int) rand());
|
||||
} // end for
|
||||
|
||||
tmp_row_.count_ = COL_NUM;
|
||||
tmp_row_.cells_ = tmp_objs_;
|
||||
}
|
||||
|
||||
ThreadContext::~ThreadContext()
|
||||
{
|
||||
}
|
||||
|
||||
int ThreadContext::add_result(ObIntermResultInfo &ir_info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObIntermResult *ir = NULL;
|
||||
int64_t expire_time = ::oceanbase::common::ObTimeUtility::current_time() + 2000000;
|
||||
if (OB_SUCCESS != (ret = alloc_result(ir))) {
|
||||
//_OB_LOG(WARN, "stress, fail to alloc result, ret:%d, ir_pool remain:%ld", ret, ir_pool_->remain());
|
||||
}
|
||||
else if (OB_SUCCESS != (ret = ir_manager_->add_result(ir_info, ir, expire_time))) {
|
||||
free_result(ir);
|
||||
} else {
|
||||
//empty
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ThreadContext::read_result(ObIntermResultInfo &ir_info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObIntermResultIterator iter;
|
||||
if (OB_SUCCESS != (ret = ir_manager_->get_result(ir_info, iter))) {
|
||||
//empty
|
||||
} else {
|
||||
//empty
|
||||
}
|
||||
|
||||
ObIIntermResultItem *ir_item = NULL;
|
||||
ObScanner scanner;
|
||||
ObScanner::Iterator scanner_iter;
|
||||
bool has_got_first_scanner = false;
|
||||
int64_t cur_row_num = 0;
|
||||
while (OB_SUCC(ret)) {
|
||||
bool should_get_next_item = false;
|
||||
if (!has_got_first_scanner) {
|
||||
should_get_next_item = true;
|
||||
} else {
|
||||
if (OB_FAIL(scanner_iter.get_next_row(tmp_row_))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
should_get_next_item = true;
|
||||
}
|
||||
} else {
|
||||
cur_row_num++;
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret) && should_get_next_item) {
|
||||
if (OB_FAIL(iter.get_next_interm_result_item(ir_item))) {
|
||||
} else {
|
||||
EXPECT_TRUE(NULL != ir_item);
|
||||
scanner.reset();
|
||||
if (!scanner.is_inited() && OB_FAIL(scanner.init())) {
|
||||
} else if (OB_FAIL(static_cast<ObIntermResultItem *>(ir_item)->to_scanner(scanner))) {
|
||||
} else {
|
||||
scanner_iter = scanner.begin();
|
||||
has_got_first_scanner = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
EXPECT_EQ(ret, OB_ITER_END);
|
||||
EXPECT_EQ(cur_row_num, 10000);
|
||||
if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ThreadContext::read_and_delete_result(ObIntermResultInfo &ir_info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObIntermResultIterator iter;
|
||||
if (OB_SUCCESS != (ret = ir_manager_->get_result(ir_info, iter))) {
|
||||
//empty
|
||||
} else {
|
||||
//empty
|
||||
}
|
||||
|
||||
ObIIntermResultItem *ir_item = NULL;
|
||||
ObScanner scanner;
|
||||
ObScanner::Iterator scanner_iter;
|
||||
bool has_got_first_scanner = false;
|
||||
int64_t cur_row_num = 0;
|
||||
while (OB_SUCC(ret)) {
|
||||
bool should_get_next_item = false;
|
||||
if (!has_got_first_scanner) {
|
||||
should_get_next_item = true;
|
||||
} else {
|
||||
if (OB_FAIL(scanner_iter.get_next_row(tmp_row_))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
should_get_next_item = true;
|
||||
}
|
||||
} else {
|
||||
cur_row_num++;
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret) && should_get_next_item) {
|
||||
if (OB_FAIL(iter.get_next_interm_result_item(ir_item))) {
|
||||
} else {
|
||||
EXPECT_TRUE(NULL != ir_item);
|
||||
scanner.reset();
|
||||
if (!scanner.is_inited() && OB_FAIL(scanner.init())) {
|
||||
} else if (OB_FAIL(static_cast<ObIntermResultItem *>(ir_item)->to_scanner(scanner))) {
|
||||
} else {
|
||||
scanner_iter = scanner.begin();
|
||||
has_got_first_scanner = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
EXPECT_EQ(ret, OB_ITER_END);
|
||||
EXPECT_EQ(cur_row_num, 10000);
|
||||
if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
|
||||
if (OB_FAIL(ret)) {
|
||||
//empty
|
||||
} else {
|
||||
ret = ir_manager_->delete_result(iter);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ThreadContext::alloc_result(ObIntermResult *&interm_result)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObIntermResult *ir = NULL;
|
||||
|
||||
// add row
|
||||
if (OB_SUCCESS != (ret = ir_manager_->alloc_result(ir))) {
|
||||
//_OB_LOG(WARN, "stress, fail to alloc interm result, ret:%d, ir_pool remain:%ld", ret, ir_pool_->remain());
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < 10000; ++i) {
|
||||
ret = ir->add_row(OB_SYS_TENANT_ID, row_);
|
||||
}
|
||||
if (OB_FAIL(ret))
|
||||
{
|
||||
ir_manager_->free_result(ir);
|
||||
ir = NULL;
|
||||
} else {
|
||||
//empty
|
||||
}
|
||||
interm_result = ir;
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ThreadContext::free_result(ObIntermResult *interm_result)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_SUCCESS != (ret = ir_manager_->free_result(interm_result))) {
|
||||
_OB_LOG(ERROR, "free ir fail");
|
||||
} else {
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
class StressRunner : public share::ObThreadPool
|
||||
{
|
||||
public:
|
||||
static const int64_t IR_INFO_COUNT = 1024;
|
||||
static const int64_t THREAD_COUNT = 16;
|
||||
|
||||
StressRunner();
|
||||
virtual ~StressRunner();
|
||||
|
||||
void run1();
|
||||
|
||||
volatile bool running_;
|
||||
volatile bool add_;
|
||||
volatile bool read_;
|
||||
volatile bool read_and_delete_;
|
||||
private:
|
||||
// disallow copy
|
||||
StressRunner(const StressRunner &other);
|
||||
StressRunner& operator=(const StressRunner &ohter);
|
||||
private:
|
||||
int64_t last_time_;
|
||||
ObIntermResultInfo ir_info_[IR_INFO_COUNT];
|
||||
ThreadContext tc_[THREAD_COUNT];
|
||||
};
|
||||
|
||||
StressRunner::StressRunner() : running_(true), add_(true), read_(true), read_and_delete_(true)
|
||||
{
|
||||
srand((unsigned int)time(NULL));
|
||||
last_time_ = -1;
|
||||
ObAddr server;
|
||||
server.set_ip_addr("127.0.0.1", 8888);
|
||||
|
||||
int64_t i = 0;
|
||||
for (; i < IR_INFO_COUNT / 2; ++i) {
|
||||
ObSliceID slice_id;
|
||||
slice_id.set_server(server);
|
||||
slice_id.set_execution_id(1);
|
||||
slice_id.set_job_id(i + 1);
|
||||
slice_id.set_task_id(i + 2);
|
||||
slice_id.set_slice_id(i + 3);
|
||||
ir_info_[i].init(slice_id);
|
||||
}
|
||||
for (; i < IR_INFO_COUNT; ++i) {
|
||||
ObSliceID slice_id;
|
||||
slice_id.set_server(server);
|
||||
slice_id.set_execution_id(1);
|
||||
slice_id.set_job_id(i + 1);
|
||||
slice_id.set_task_id(i + 1);
|
||||
slice_id.set_slice_id(i + 1);
|
||||
ir_info_[i].init(slice_id);
|
||||
}
|
||||
set_thread_count(THREAD_COUNT);
|
||||
}
|
||||
|
||||
StressRunner::~StressRunner()
|
||||
{
|
||||
}
|
||||
|
||||
void StressRunner::run1()
|
||||
{
|
||||
long tc_num = (long) get_thread_idx();
|
||||
while(running_) {
|
||||
if (0 == tc_num) {
|
||||
int64_t cur_time = ::oceanbase::common::ObTimeUtility::current_time();
|
||||
if (last_time_ == -1 || cur_time - last_time_ > 1000000)
|
||||
{
|
||||
last_time_ = cur_time;
|
||||
ob_print_mod_memory_usage();
|
||||
}
|
||||
} else {
|
||||
//_OB_LOG(WARN, "tc_num=%ld, add_=%d", tc_num, (int) add_);
|
||||
int64_t idx = rand() % IR_INFO_COUNT;
|
||||
int64_t op = rand() % 6;
|
||||
if (0 <= op && 2 >= op && read_) {
|
||||
tc_[tc_num].read_result(ir_info_[idx]);
|
||||
} else if (3 <= op && 4 >= op && add_) {
|
||||
int ret = tc_[tc_num].add_result(ir_info_[idx]);
|
||||
if (OB_SUCC(ret)) {
|
||||
//_OB_LOG(WARN, "add result succ");
|
||||
}
|
||||
} else if (5 == op && read_and_delete_) {
|
||||
tc_[tc_num].read_and_delete_result(ir_info_[idx]);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class ObIntermResultManagerTest : public ::testing::Test
|
||||
{
|
||||
public:
|
||||
StressRunner runner_;
|
||||
|
||||
ObIntermResultManagerTest();
|
||||
virtual ~ObIntermResultManagerTest();
|
||||
virtual void SetUp();
|
||||
virtual void TearDown();
|
||||
private:
|
||||
// disallow copy
|
||||
ObIntermResultManagerTest(const ObIntermResultManagerTest &other);
|
||||
ObIntermResultManagerTest& operator=(const ObIntermResultManagerTest &other);
|
||||
private:
|
||||
};
|
||||
|
||||
ObIntermResultManagerTest::ObIntermResultManagerTest()
|
||||
{
|
||||
}
|
||||
|
||||
ObIntermResultManagerTest::~ObIntermResultManagerTest()
|
||||
{
|
||||
}
|
||||
|
||||
void ObIntermResultManagerTest::SetUp()
|
||||
{
|
||||
}
|
||||
|
||||
void ObIntermResultManagerTest::TearDown()
|
||||
{
|
||||
}
|
||||
|
||||
TEST_F(ObIntermResultManagerTest, basic_test)
|
||||
{
|
||||
//runner_.add_ = false;
|
||||
//runner_.read_ = false;
|
||||
//runner_.read_and_delete_ = false;
|
||||
runner_.start();
|
||||
while(true) {
|
||||
sleep(30);
|
||||
runner_.read_ = false;
|
||||
runner_.read_and_delete_ = false;
|
||||
runner_.add_ = false;
|
||||
while(true) {
|
||||
sleep(30);
|
||||
runner_.add_ = true;
|
||||
sleep(30);
|
||||
runner_.read_ = true;
|
||||
sleep(30);
|
||||
runner_.read_and_delete_ = true;
|
||||
sleep(30);
|
||||
runner_.add_ = false;
|
||||
runner_.read_ = false;
|
||||
runner_.read_and_delete_ = false;
|
||||
}
|
||||
}
|
||||
runner_.stop();
|
||||
runner_.wait();
|
||||
}
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
oceanbase::sql::init_sql_executor_singletons();
|
||||
oceanbase::common::ObLogger::get_logger().set_log_level("INFO");
|
||||
::testing::InitGoogleTest(&argc,argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
||||
207
unittest/sql/executor/stress_scanner_pool.cpp
Normal file
207
unittest/sql/executor/stress_scanner_pool.cpp
Normal file
@ -0,0 +1,207 @@
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
#include "share/ob_thread_pool.h"
|
||||
#include "sql/executor/ob_interm_result_manager.h"
|
||||
#include "sql/executor/ob_interm_result_pool.h"
|
||||
#include "sql/ob_sql_init.h"
|
||||
#include <stdlib.h>
|
||||
#include <math.h>
|
||||
#include <time.h>
|
||||
|
||||
using namespace oceanbase;
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
|
||||
class ThreadContext
|
||||
{
|
||||
public:
|
||||
static const int64_t COL_NUM = 16;
|
||||
static const int64_t MEM_LIMIT = 2147483648;
|
||||
|
||||
ThreadContext();
|
||||
virtual ~ThreadContext();
|
||||
|
||||
int alloc_scanner(ObScanner *&scanner);
|
||||
int free_scanner(ObScanner *scanner);
|
||||
private:
|
||||
ObScannerPool* scanner_pool_;
|
||||
oceanbase::common::ObNewRow row_;
|
||||
oceanbase::common::ObNewRow tmp_row_;
|
||||
ObObj objs_[COL_NUM];
|
||||
ObObj tmp_objs_[COL_NUM];
|
||||
};
|
||||
|
||||
ThreadContext::ThreadContext()
|
||||
{
|
||||
scanner_pool_ = ObScannerPool::get_instance();
|
||||
|
||||
row_.count_ = COL_NUM;
|
||||
row_.cells_ = objs_;
|
||||
for (int64_t i = 0; i < COL_NUM; ++i) {
|
||||
row_.cells_[i].set_int((int) rand());
|
||||
} // end for
|
||||
|
||||
tmp_row_.count_ = COL_NUM;
|
||||
tmp_row_.cells_ = tmp_objs_;
|
||||
}
|
||||
|
||||
ThreadContext::~ThreadContext()
|
||||
{
|
||||
}
|
||||
|
||||
int ThreadContext::alloc_scanner(ObScanner *&scanner)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_SUCCESS != (ret = scanner_pool_->alloc_scanner(OB_SYS_TENANT_ID, ThreadContext::MEM_LIMIT, scanner)))
|
||||
{
|
||||
_OB_LOG(WARN, "stress, fail to alloc scanner, ret:%d, scanner_pool remain:%ld", ret, scanner_pool_->remain());
|
||||
} else {
|
||||
//_OB_LOG(INFO, "succ alloc");
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
//empty
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ThreadContext::free_scanner(ObScanner *scanner)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
scanner->reuse();
|
||||
if (OB_SUCCESS != (ret = scanner_pool_->free_scanner(scanner))) {
|
||||
_OB_LOG(ERROR, "free scanner fail");
|
||||
} else {
|
||||
//empty
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
class StressRunner : public share::ObThreadPool
|
||||
{
|
||||
public:
|
||||
static const int64_t IR_INFO_COUNT = 1024;
|
||||
static const int64_t THREAD_COUNT = 16;
|
||||
|
||||
StressRunner();
|
||||
virtual ~StressRunner();
|
||||
|
||||
void run1();
|
||||
|
||||
volatile bool running_;
|
||||
volatile bool alloc_;
|
||||
volatile bool free_;
|
||||
private:
|
||||
// disallow copy
|
||||
StressRunner(const StressRunner &other);
|
||||
StressRunner& operator=(const StressRunner &ohter);
|
||||
private:
|
||||
int64_t last_time_;
|
||||
ThreadContext tc_[THREAD_COUNT];
|
||||
};
|
||||
|
||||
StressRunner::StressRunner() : running_(true), alloc_(true), free_(true)
|
||||
{
|
||||
srand((unsigned int)time(NULL));
|
||||
last_time_ = -1;
|
||||
set_thread_count(THREAD_COUNT);
|
||||
}
|
||||
|
||||
StressRunner::~StressRunner()
|
||||
{
|
||||
}
|
||||
|
||||
void StressRunner::run1()
|
||||
{
|
||||
|
||||
long tc_num = (long) get_thread_idx();
|
||||
while(running_) {
|
||||
if (0 == tc_num) {
|
||||
int64_t cur_time = ::oceanbase::common::ObTimeUtility::current_time();
|
||||
if (last_time_ == -1 || cur_time - last_time_ > 1000000) {
|
||||
last_time_ = cur_time;
|
||||
ob_print_mod_memory_usage();
|
||||
}
|
||||
} else {
|
||||
//_OB_LOG(WARN, "tc_num=%ld, add_=%d", tc_num, (int) add_);
|
||||
ObScanner *scanner = NULL;
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_SUCCESS != (ret = tc_[tc_num].alloc_scanner(scanner))) {
|
||||
//_OB_LOG(WARN, "alloc fail");
|
||||
} else {
|
||||
scanner->reuse();
|
||||
ret = tc_[tc_num].free_scanner(scanner);
|
||||
if (OB_FAIL(ret)) {
|
||||
_OB_LOG(ERROR, "free scanner fail, ret=%d", ret);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class ObIntermResultManagerTest : public ::testing::Test
|
||||
{
|
||||
public:
|
||||
StressRunner runner_;
|
||||
|
||||
ObIntermResultManagerTest();
|
||||
virtual ~ObIntermResultManagerTest();
|
||||
virtual void SetUp();
|
||||
virtual void TearDown();
|
||||
private:
|
||||
// disallow copy
|
||||
ObIntermResultManagerTest(const ObIntermResultManagerTest &other);
|
||||
ObIntermResultManagerTest& operator=(const ObIntermResultManagerTest &other);
|
||||
private:
|
||||
};
|
||||
|
||||
ObIntermResultManagerTest::ObIntermResultManagerTest()
|
||||
{
|
||||
}
|
||||
|
||||
ObIntermResultManagerTest::~ObIntermResultManagerTest()
|
||||
{
|
||||
}
|
||||
|
||||
void ObIntermResultManagerTest::SetUp()
|
||||
{
|
||||
}
|
||||
|
||||
void ObIntermResultManagerTest::TearDown()
|
||||
{
|
||||
}
|
||||
|
||||
TEST_F(ObIntermResultManagerTest, basic_test)
|
||||
{
|
||||
runner_.start();
|
||||
while(true) {
|
||||
sleep(15);
|
||||
runner_.alloc_ = false;
|
||||
sleep(15);
|
||||
runner_.free_ = false;
|
||||
sleep(15);
|
||||
runner_.alloc_ = true;
|
||||
sleep(15);
|
||||
runner_.free_ = true;
|
||||
}
|
||||
runner_.stop();
|
||||
runner_.wait();
|
||||
}
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
oceanbase::sql::init_sql_executor_singletons();
|
||||
oceanbase::common::ObLogger::get_logger().set_log_level("INFO");
|
||||
::testing::InitGoogleTest(&argc,argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
||||
176
unittest/sql/executor/test_disk_interm_result.cpp
Normal file
176
unittest/sql/executor/test_disk_interm_result.cpp
Normal file
@ -0,0 +1,176 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
#include "lib/alloc/ob_malloc_allocator.h"
|
||||
#include "storage/blocksstable/ob_data_file_prepare.h"
|
||||
#include "storage/blocksstable/ob_macro_file.h"
|
||||
#include "sql/executor/ob_interm_result.h"
|
||||
#include "sql/executor/ob_interm_result_pool.h"
|
||||
#include "sql/executor/ob_interm_result_manager.h"
|
||||
#include "sql/executor/ob_interm_result_item.h"
|
||||
#include "share/config/ob_server_config.h"
|
||||
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace sql
|
||||
{
|
||||
using namespace common;
|
||||
|
||||
class TestDiskIntermResult : public blocksstable::TestDataFilePrepare
|
||||
{
|
||||
public:
|
||||
TestDiskIntermResult() : blocksstable::TestDataFilePrepare("TestDiskIR", 2<<20, 1000)
|
||||
{
|
||||
}
|
||||
|
||||
virtual void SetUp() override
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
blocksstable::TestDataFilePrepare::SetUp();
|
||||
ret = blocksstable::ObMacroFileManager::get_instance().init();
|
||||
ASSERT_EQ(OB_SUCCESS, ret);
|
||||
|
||||
GCONF.enable_sql_operator_dump.set_value("True");
|
||||
}
|
||||
|
||||
virtual void TearDown() override
|
||||
{
|
||||
blocksstable::TestDataFilePrepare::TearDown();
|
||||
}
|
||||
|
||||
protected:
|
||||
};
|
||||
|
||||
TEST_F(TestDiskIntermResult, disk_write_read)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
ObObj cells[2];
|
||||
cells[0].set_int(1);
|
||||
const char *str = __FILE__;
|
||||
cells[1].set_varchar(str, (int32_t)strlen(str));
|
||||
|
||||
ObNewRow row;
|
||||
row.count_ = 2;
|
||||
row.cells_ = cells;
|
||||
|
||||
ObIntermResult *ir = NULL;
|
||||
ret = ObIntermResultManager::get_instance()->alloc_result(ir);
|
||||
ASSERT_EQ(OB_SUCCESS, ret);
|
||||
lib::ObMallocAllocator *malloc_allocator = lib::ObMallocAllocator::get_instance();
|
||||
ret = malloc_allocator->create_tenant_ctx_allocator(OB_SYS_TENANT_ID);
|
||||
ASSERT_EQ(OB_SUCCESS, ret);
|
||||
ret = malloc_allocator->create_tenant_ctx_allocator(OB_SYS_TENANT_ID, common::ObCtxIds::WORK_AREA);
|
||||
ASSERT_EQ(OB_SUCCESS, ret);
|
||||
malloc_allocator->set_tenant_limit(OB_SYS_TENANT_ID, 1L << 30);
|
||||
ASSERT_EQ(OB_SUCCESS, ret);
|
||||
// 50MB for work area
|
||||
ret = lib::set_wa_limit(OB_SYS_TENANT_ID, 5);
|
||||
ASSERT_EQ(OB_SUCCESS, ret);
|
||||
|
||||
// write 100MB
|
||||
static int64_t max_row_cnt = 1000000000;
|
||||
int64_t row_cnt = 0;
|
||||
for (;row_cnt < max_row_cnt; row_cnt++) {
|
||||
ret = ir->add_row(OB_SYS_TENANT_ID, row);
|
||||
ASSERT_EQ(OB_SUCCESS, ret);
|
||||
if (row_cnt % 10000 == 0) {
|
||||
int64_t size = 0;
|
||||
ret = ir->get_all_data_size(size);
|
||||
ASSERT_EQ(OB_SUCCESS, ret);
|
||||
if (size > (100L << 20)) {
|
||||
row_cnt++;
|
||||
break;
|
||||
} else {
|
||||
LOG_INFO("write progress", K(row_cnt), K(size));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ret = ir->complete_add_rows(OB_SYS_TENANT_ID);
|
||||
ASSERT_EQ(OB_SUCCESS, ret);
|
||||
|
||||
int64_t mem_size = 0;
|
||||
int64_t disk_size = 0;
|
||||
ret = ir->get_data_size_detail(mem_size, disk_size);
|
||||
ASSERT_GT(mem_size + disk_size, 100L << 20);
|
||||
ASSERT_GT(mem_size, 0);
|
||||
ASSERT_GT(disk_size, 0);
|
||||
|
||||
ObAddr server;
|
||||
server.set_ip_addr("127.0.0.1", 80);
|
||||
ObSliceID slice_id;
|
||||
slice_id.set_server(server);
|
||||
slice_id.set_execution_id(1);
|
||||
slice_id.set_job_id(1);
|
||||
slice_id.set_task_id(1);
|
||||
slice_id.set_slice_id(1);
|
||||
ObIntermResultInfo iraddr;
|
||||
iraddr.init(slice_id);
|
||||
|
||||
ret = ObIntermResultManager::get_instance()->add_result(iraddr, ir, 1000000000L);
|
||||
ASSERT_EQ(OB_SUCCESS, ret);
|
||||
|
||||
ObIntermResultIterator iter;
|
||||
ret = ObIntermResultManager::get_instance()->get_result(iraddr, iter);
|
||||
ASSERT_EQ(OB_SUCCESS, ret);
|
||||
|
||||
char *buf = new char[ObScanner::DEFAULT_MAX_SERIALIZE_SIZE * 2];
|
||||
int64_t read_row_cnt = 0;
|
||||
ObScanner scanner;
|
||||
while (true) {
|
||||
ObIIntermResultItem *it = NULL;
|
||||
ret = iter.get_next_interm_result_item(it);
|
||||
if (OB_ITER_END == ret) {
|
||||
break;
|
||||
}
|
||||
ASSERT_EQ(OB_SUCCESS, ret);
|
||||
ASSERT_TRUE(it != NULL);
|
||||
|
||||
int64_t len = it->get_data_len();
|
||||
ret = it->copy_data(buf, len);
|
||||
ASSERT_EQ(OB_SUCCESS, ret);
|
||||
|
||||
int64_t pos = 0;
|
||||
ret = scanner.deserialize(buf, len, pos);
|
||||
ASSERT_EQ(OB_SUCCESS, ret);
|
||||
|
||||
ObScanner::Iterator row_iter = scanner.begin();
|
||||
while (OB_SUCCESS == row_iter.get_next_row(row)) {
|
||||
read_row_cnt++;
|
||||
}
|
||||
}
|
||||
|
||||
ASSERT_EQ(row_cnt, read_row_cnt);
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
void ignore_sig(int sig)
|
||||
{
|
||||
UNUSED(sig);
|
||||
}
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
signal(49, ignore_sig);
|
||||
oceanbase::common::ObLogger::get_logger().set_log_level("INFO");
|
||||
oceanbase::sql::ObIntermResultItemPool::build_instance();
|
||||
oceanbase::sql::ObIntermResultPool::build_instance();
|
||||
oceanbase::sql::ObIntermResultManager::build_instance();
|
||||
testing::InitGoogleTest(&argc, argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
||||
484
unittest/sql/executor/test_distributed_scheduler.cpp
Normal file
484
unittest/sql/executor/test_distributed_scheduler.cpp
Normal file
@ -0,0 +1,484 @@
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
#include <stdarg.h>
|
||||
#include "sql/executor/ob_distributed_scheduler.h"
|
||||
#include "ob_mock_utils.h"
|
||||
#include "sql/engine/table/ob_table_scan.h"
|
||||
#include "sql/engine/sort/ob_sort.h"
|
||||
#include "sql/executor/ob_root_transmit.h"
|
||||
#include "sql/executor/ob_distributed_transmit.h"
|
||||
#include "sql/executor/ob_direct_receive.h"
|
||||
#include "sql/executor/ob_fifo_receive.h"
|
||||
#include "sql/executor/ob_distributed_job_control.h"
|
||||
#include "sql/executor/ob_job_parser.h"
|
||||
#include "sql/executor/ob_task_spliter_factory.h"
|
||||
#include "sql/engine/ob_physical_plan_ctx.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/ob_sql_init.h"
|
||||
#include "lib/utility/ob_tracepoint.h"
|
||||
|
||||
using namespace oceanbase;
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
using namespace oceanbase::storage;
|
||||
using namespace oceanbase::share;
|
||||
using namespace oceanbase::share::schema;
|
||||
|
||||
#define TEST_FAIL(statement) \
|
||||
do { \
|
||||
if (OB_SUCCESS == ret && OB_SUCCESS != (ret = (statement))) { \
|
||||
return ret; \
|
||||
} \
|
||||
} while(0)
|
||||
|
||||
class ObDistributedSchedulerTest : public ::testing::Test
|
||||
{
|
||||
public:
|
||||
|
||||
const static int64_t TEST_SPLIT_TASK_COUNT = 7;
|
||||
const static int64_t TEST_PARA_SERVER_COUNT = 2;
|
||||
const static int64_t TEST_SERVER_PARA_THREAD_COUNT = 3;
|
||||
|
||||
const static int64_t TEST_TABLE_ID = 3003;
|
||||
const static int64_t TEST_INDEX_ID = 3004;
|
||||
const static uint64_t COLUMN_ID_1 = 16;
|
||||
const static uint64_t COLUMN_ID_2 = 17;
|
||||
|
||||
const static int64_t TEST_LIMIT = 10;
|
||||
const static int64_t TEST_OFFSET = 0;
|
||||
|
||||
ObDistributedSchedulerTest();
|
||||
virtual ~ObDistributedSchedulerTest();
|
||||
virtual void SetUp();
|
||||
virtual void TearDown();
|
||||
|
||||
ObMockSqlExecutorRpc rpc_;
|
||||
ObPhysicalPlan *phy_plan_;
|
||||
|
||||
int create_plan_tree(ObExecContext &ctx);
|
||||
int exception_test();
|
||||
private:
|
||||
// disallow copy
|
||||
ObDistributedSchedulerTest(const ObDistributedSchedulerTest &other);
|
||||
ObDistributedSchedulerTest& operator=(const ObDistributedSchedulerTest &other);
|
||||
};
|
||||
ObDistributedSchedulerTest::ObDistributedSchedulerTest()
|
||||
{
|
||||
}
|
||||
|
||||
ObDistributedSchedulerTest::~ObDistributedSchedulerTest()
|
||||
{
|
||||
}
|
||||
|
||||
void ObDistributedSchedulerTest::SetUp()
|
||||
{
|
||||
}
|
||||
|
||||
int ObDistributedSchedulerTest::create_plan_tree(ObExecContext &ctx)
|
||||
{
|
||||
UNUSED(ctx);
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
phy_plan_ = ObPhysicalPlan::alloc();
|
||||
ObPhyOperator *cur_op = NULL;
|
||||
ObPhyOperator *tmp_op = NULL;
|
||||
int err_code = OB_SUCCESS;
|
||||
|
||||
ObSqlExpression *limit_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(phy_plan_, limit_expr));
|
||||
EXPECT_FALSE(NULL == limit_expr);
|
||||
ObPostExprItem limit_expr_item;
|
||||
limit_expr_item.set_int(TEST_LIMIT);
|
||||
limit_expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, limit_expr->add_expr_item(limit_expr_item));
|
||||
|
||||
ObSqlExpression *offset_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(phy_plan_, offset_expr));
|
||||
EXPECT_FALSE(NULL == offset_expr);
|
||||
ObPostExprItem offset_expr_item;
|
||||
offset_expr_item.set_int(TEST_OFFSET);
|
||||
offset_expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, offset_expr->add_expr_item(offset_expr_item));
|
||||
|
||||
/*
|
||||
* calculate c0 % TEST_SPLIT_TASK_COUNT
|
||||
* */
|
||||
ObSqlExpression *hash_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(phy_plan_, hash_expr));
|
||||
EXPECT_FALSE(NULL == hash_expr);
|
||||
ObPostExprItem expr_item;
|
||||
expr_item.set_column(0);
|
||||
EXPECT_EQ(OB_SUCCESS, hash_expr->add_expr_item(expr_item));
|
||||
expr_item.set_int(TEST_SPLIT_TASK_COUNT);
|
||||
expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, hash_expr->add_expr_item(expr_item));
|
||||
expr_item.set_op("%", 2);
|
||||
EXPECT_EQ(OB_SUCCESS, hash_expr->add_expr_item(expr_item));
|
||||
|
||||
/*
|
||||
* calculate c0 % 1
|
||||
* */
|
||||
ObSqlExpression *iden_hash_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(phy_plan_, iden_hash_expr));
|
||||
EXPECT_FALSE(NULL == iden_hash_expr);
|
||||
ObPostExprItem iden_expr_item;
|
||||
iden_expr_item.set_column(0);
|
||||
EXPECT_EQ(OB_SUCCESS, iden_hash_expr->add_expr_item(iden_expr_item));
|
||||
iden_expr_item.set_int(1);
|
||||
iden_expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, iden_hash_expr->add_expr_item(iden_expr_item));
|
||||
iden_expr_item.set_op("%", 2);
|
||||
EXPECT_EQ(OB_SUCCESS, iden_hash_expr->add_expr_item(iden_expr_item));
|
||||
|
||||
|
||||
ObPhysicalPlanCtx *plan_ctx = ctx.get_physical_plan_ctx();
|
||||
common::ObIArray<common::ObObj> ¶m_store = plan_ctx->get_param_store();
|
||||
ObObj value1;
|
||||
value1.set_int(3);
|
||||
param_store.push_back(value1);
|
||||
ObObj value2;
|
||||
value2.set_int(1);
|
||||
param_store.push_back(value2);
|
||||
ObObj value3;
|
||||
value3.set_int(2);
|
||||
param_store.push_back(value3);
|
||||
|
||||
|
||||
/*
|
||||
* calculate a + ?
|
||||
* */
|
||||
ObSqlExpression *partition_func = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(phy_plan_, partition_func));
|
||||
EXPECT_FALSE(NULL == partition_func);
|
||||
ObPostExprItem partition_func_item;
|
||||
partition_func_item.set_column(0);
|
||||
EXPECT_EQ(OB_SUCCESS, partition_func->add_expr_item(partition_func_item));
|
||||
ObObj index_value;
|
||||
index_value.set_int(2);
|
||||
partition_func_item.assign(T_QUESTIONMARK, index_value);
|
||||
EXPECT_EQ(OB_SUCCESS, partition_func->add_expr_item(partition_func_item));
|
||||
partition_func_item.set_op("+", 2);
|
||||
EXPECT_EQ(OB_SUCCESS, partition_func->add_expr_item(partition_func_item));
|
||||
|
||||
ObColumnRefRawExpr ref_col1(TEST_TABLE_ID, COLUMN_ID_1, T_REF_COLUMN);
|
||||
//ObArray<ColumnItem> single_range_columns;
|
||||
ColumnItem col1;
|
||||
col1.column_id_ = COLUMN_ID_1;
|
||||
col1.table_id_ = TEST_TABLE_ID;
|
||||
col1.data_type_ = ObIntType;
|
||||
col1.column_name_ = ObString::make_string("a");
|
||||
//EXPECT_EQ(OB_SUCCESS, single_range_columns.push_back(col1));
|
||||
ref_col1.add_flag(IS_COLUMN);
|
||||
ObColumnRefRawExpr ref_col2(TEST_TABLE_ID, COLUMN_ID_2, T_REF_COLUMN);
|
||||
ColumnItem col2;
|
||||
col2.column_id_ = COLUMN_ID_2;
|
||||
col2.table_id_ = TEST_TABLE_ID;
|
||||
col2.data_type_ = ObIntType;
|
||||
col2.column_name_ = ObString::make_string("b");
|
||||
//EXPECT_EQ(OB_SUCCESS, single_range_columns.push_back(col2));
|
||||
ref_col2.add_flag(IS_COLUMN);
|
||||
|
||||
// 构造 (a = ?)
|
||||
ObObj index1;
|
||||
index1.set_unknown(0);
|
||||
ObConstRawExpr const_col1(index1, T_QUESTIONMARK);
|
||||
const_col1.add_flag(IS_STATIC_PARAM);
|
||||
ObOpRawExpr condition1(&ref_col1, &const_col1, T_OP_EQ); // a = ?构造完毕
|
||||
// 构造 (b > ?)
|
||||
ObObj index2;
|
||||
index2.set_unknown(1);
|
||||
ObConstRawExpr const_col2(index2, T_QUESTIONMARK);
|
||||
const_col2.add_flag(IS_STATIC_PARAM);
|
||||
ObOpRawExpr condition2(&ref_col2, &const_col2, T_OP_GT); // b > ?构造完毕
|
||||
ObOpRawExpr condition3(&condition1, &condition2, T_OP_AND); // a = ? and b > ?构造完毕
|
||||
|
||||
ObArray<ColumnItem> scan_range_columns;
|
||||
EXPECT_EQ(OB_SUCCESS, scan_range_columns.push_back(col2));
|
||||
ObArray<ColumnItem> partition_range_columns;
|
||||
EXPECT_EQ(OB_SUCCESS, partition_range_columns.push_back(col1));
|
||||
ObQueryRange *query_range = OB_NEW(ObQueryRange, ObModIds::TEST);
|
||||
EXPECT_EQ(OB_SUCCESS, query_range->preliminary_extract_query_range(scan_range_columns, &condition3));
|
||||
EXPECT_EQ(OB_SUCCESS, query_range->preliminary_extract_query_range(partition_range_columns, &condition3));
|
||||
|
||||
ASSERT_EQ(OB_SUCCESS, phy_plan_->alloc_operator_by_type(PHY_TABLE_SCAN, tmp_op));
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_table_id(TEST_TABLE_ID);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_ref_table_id(TEST_TABLE_ID);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_index_table_id(TEST_INDEX_ID);
|
||||
static_cast<ObTableScan*>(tmp_op)->add_range_column(COLUMN_ID_2);
|
||||
static_cast<ObTableScan*>(tmp_op)->add_output_column(COLUMN_ID_1);
|
||||
static_cast<ObTableScan*>(tmp_op)->add_output_column(COLUMN_ID_2);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_limit_offset(*limit_expr, *offset_expr);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_query_range(*query_range);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
ASSERT_EQ(OB_SUCCESS, phy_plan_->alloc_operator_by_type(PHY_DISTRIBUTED_TRANSMIT, tmp_op));
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_split_task_count(TEST_SPLIT_TASK_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_parallel_server_count(TEST_PARA_SERVER_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_server_parallel_thread_count(TEST_SERVER_PARA_THREAD_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_shuffle_func(hash_expr);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
ASSERT_EQ(OB_SUCCESS, phy_plan_->alloc_operator_by_type(PHY_FIFO_RECEIVE, tmp_op));
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
ASSERT_EQ(OB_SUCCESS, phy_plan_->alloc_operator_by_type(PHY_DISTRIBUTED_TRANSMIT, tmp_op));
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::INTERM_SPLIT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_split_task_count(TEST_SPLIT_TASK_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_parallel_server_count(TEST_PARA_SERVER_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_server_parallel_thread_count(TEST_SERVER_PARA_THREAD_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_shuffle_func(iden_hash_expr);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
ASSERT_EQ(OB_SUCCESS, phy_plan_->alloc_operator_by_type(PHY_FIFO_RECEIVE, tmp_op));
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
ASSERT_EQ(OB_SUCCESS, phy_plan_->alloc_operator_by_type(PHY_ROOT_TRANSMIT, tmp_op));
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_split_task_count(TEST_SPLIT_TASK_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_parallel_server_count(TEST_PARA_SERVER_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_server_parallel_thread_count(TEST_SERVER_PARA_THREAD_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_shuffle_func(hash_expr);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
phy_plan_->set_main_query(cur_op);
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObDistributedSchedulerTest::TearDown()
|
||||
{
|
||||
}
|
||||
|
||||
int ObDistributedSchedulerTest::exception_test()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t remain_time_us = 1000 * 1000;
|
||||
uint64_t query_id = OB_INVALID_ID;
|
||||
ObDistributedSchedulerManager *sc_manager = ObDistributedSchedulerManager::get_instance();
|
||||
|
||||
ObAddr server;
|
||||
server.set_ip_addr("127.0.0.1", 8888);
|
||||
ObArenaAllocator allocator(ObModIds::OB_SQL_EXEC_CONTEXT);
|
||||
ObPhyTableLocationArray table_locs(allocator);
|
||||
ObPhyTableLocation table_loc;
|
||||
ObPartitionLocation partition_loc;
|
||||
partition_loc.set_table_id(TEST_TABLE_ID);
|
||||
partition_loc.set_partition_id(9);
|
||||
table_loc.set_table_id(TEST_TABLE_ID);
|
||||
EXPECT_EQ(OB_SUCCESS, table_loc.add_partition_location(partition_loc));
|
||||
ObMockFetchResultStreamHandle resp_handler(allocator);
|
||||
ObExecuteResult exe_result;
|
||||
ObExecContext exec_ctx;
|
||||
TEST_FAIL(exec_ctx.init_phy_op(100));
|
||||
TEST_FAIL(exec_ctx.create_physical_plan_ctx());
|
||||
ObTaskExecutorCtx *executor_ctx = exec_ctx.get_task_executor_ctx();
|
||||
executor_ctx->set_task_executor_rpc(&rpc_);
|
||||
executor_ctx->set_server(server);
|
||||
executor_ctx->set_partition_location_cache(&rpc_.partition_loc_cache_);
|
||||
executor_ctx->set_partition_service(&rpc_.partition_service_);
|
||||
executor_ctx->set_execute_result(&exe_result);
|
||||
executor_ctx->init_table_location(1);
|
||||
EXPECT_EQ(OB_SUCCESS, table_locs.push_back(table_loc));
|
||||
executor_ctx->set_table_locations(table_locs);
|
||||
|
||||
TEST_FAIL(create_plan_tree(exec_ctx));
|
||||
|
||||
TEST_FAIL(sc_manager->alloc_scheduler(query_id, remain_time_us));
|
||||
TEST_FAIL(sc_manager->schedule(query_id, exec_ctx, phy_plan_));
|
||||
TEST_FAIL(sc_manager->free_scheduler(query_id));
|
||||
return ret;
|
||||
}
|
||||
|
||||
TEST_F(ObDistributedSchedulerTest, basic_test)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
//启动模拟收包队列
|
||||
ObMockPacketQueueThread::get_instance()->start();
|
||||
|
||||
int64_t remain_time_us = 1000 * 1000;
|
||||
uint64_t query_id = OB_INVALID_ID;
|
||||
ObDistributedSchedulerManager *sc_manager = ObDistributedSchedulerManager::get_instance();
|
||||
|
||||
ObPhyTableLocationSEArray table_locs;
|
||||
ObPhyTableLocation table_loc;
|
||||
ObPartitionLocation partition_loc;
|
||||
partition_loc.set_table_id(TEST_TABLE_ID);
|
||||
partition_loc.set_partition_id(9);
|
||||
table_loc.set_table_id(TEST_TABLE_ID);
|
||||
ASSERT_EQ(OB_SUCCESS, table_loc.add_partition_location(partition_loc));
|
||||
ASSERT_EQ(OB_SUCCESS, table_locs.push_back(table_loc));
|
||||
|
||||
ObAddr server;
|
||||
server.set_ip_addr("127.0.0.1", 8888);
|
||||
ObArenaAllocator allocator(ObModIds::OB_SQL_EXEC_CONTEXT);
|
||||
ObMockFetchResultStreamHandle resp_handler(allocator);
|
||||
ObExecuteResult exe_result;
|
||||
ObExecContext exec_ctx;
|
||||
exec_ctx.init_phy_op(100);
|
||||
exec_ctx.create_physical_plan_ctx();
|
||||
ObPhysicalPlanCtx *plan_ctx = exec_ctx.get_physical_plan_ctx();
|
||||
//plan_ctx->set_task_response_handler(resp_handler);
|
||||
plan_ctx->set_server(server);
|
||||
plan_ctx->set_timeout_timestamp(::oceanbase::common::ObTimeUtility::current_time() + 2000L * 1000L);
|
||||
ObTaskExecutorCtx *executor_ctx = exec_ctx.get_task_executor_ctx();
|
||||
executor_ctx->set_task_executor_rpc(&rpc_);
|
||||
executor_ctx->set_server(server);
|
||||
executor_ctx->set_partition_location_cache(&rpc_.partition_loc_cache_);
|
||||
executor_ctx->set_partition_service(&rpc_.partition_service_);
|
||||
executor_ctx->set_execute_result(&exe_result);
|
||||
executor_ctx->set_table_locations(table_locs);
|
||||
|
||||
create_plan_tree(exec_ctx);
|
||||
|
||||
ASSERT_EQ(OB_SUCCESS, sc_manager->alloc_scheduler(query_id, remain_time_us));
|
||||
ASSERT_EQ(OB_SUCCESS, sc_manager->schedule(query_id, exec_ctx, phy_plan_));
|
||||
ASSERT_EQ(OB_SUCCESS, sc_manager->free_scheduler(query_id));
|
||||
//ObExecutor ob_exe;
|
||||
//ASSERT_EQ(OB_SUCCESS, ob_exe.execute_plan(exec_ctx, phy_plan_));
|
||||
ASSERT_EQ(OB_SUCCESS, exe_result.open(exec_ctx));
|
||||
const ObNewRow *tmp_row = NULL;
|
||||
while(OB_SUCCESS == (ret = exe_result.get_next_row(exec_ctx, tmp_row))) {
|
||||
SQL_EXE_LOG(INFO, "get a row", K(*tmp_row));
|
||||
}
|
||||
ASSERT_EQ(OB_ITER_END, ret);
|
||||
ASSERT_EQ(OB_SUCCESS, exe_result.close(exec_ctx));
|
||||
}
|
||||
|
||||
#define EXCEPTION_TEST_1(test_name, expected_ret, file, func, key, err) \
|
||||
TEST_F(ObDistributedSchedulerTest, test_name) \
|
||||
{\
|
||||
TP_SET_ERROR(file, func, key, err); \
|
||||
int ret = exception_test(); \
|
||||
TP_SET_ERROR(file, func, key, NULL); \
|
||||
ASSERT_EQ(ret, expected_ret); \
|
||||
}
|
||||
|
||||
#define EXCEPTION_TEST_2(test_name, expected_ret, file1, func1, key1, err1, file2, func2, key2, err2) \
|
||||
TEST_F(ObDistributedSchedulerTest, test_name) \
|
||||
{\
|
||||
TP_SET_ERROR(file1, func1, key1, err1); \
|
||||
TP_SET_ERROR(file2, func2, key2, err2); \
|
||||
int ret = exception_test(); \
|
||||
TP_SET_ERROR(file1, func1, key1, NULL); \
|
||||
TP_SET_ERROR(file2, func2, key2, NULL); \
|
||||
ASSERT_EQ(ret, expected_ret); \
|
||||
}
|
||||
|
||||
#if 0
|
||||
EXCEPTION_TEST_1(et1_1, OB_NOT_INIT, "executor/ob_task.cpp", "serialize_", "t1", 1)
|
||||
EXCEPTION_TEST_1(et1_2, OB_NOT_INIT, "executor/ob_task.cpp", "deserialize_", "t1", 1)
|
||||
EXCEPTION_TEST_1(et1_3, OB_ERR_UNEXPECTED, "executor/ob_task.cpp", "deserialize_", "t2", 1)
|
||||
EXCEPTION_TEST_1(et1_4, OB_ERROR, "executor/ob_task.cpp", "serialize_tree", "t1", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_5, OB_ERROR, "executor/ob_task.cpp", "serialize_tree", "t2", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_6, OB_ERR_UNEXPECTED, "executor/ob_task.cpp", "serialize_tree", "t3", 1)
|
||||
EXCEPTION_TEST_1(et1_7, OB_ERROR, "executor/ob_task.cpp", "serialize_tree", "t4", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_8, OB_ERROR, "executor/ob_task.cpp", "deserialize_tree", "t1", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_9, OB_ALLOCATE_MEMORY_FAILED, "executor/ob_task.cpp", "deserialize_tree", "t2", 1)
|
||||
EXCEPTION_TEST_1(et1_10, OB_ERROR, "executor/ob_task.cpp", "deserialize_tree", "t3", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_11, OB_ERROR, "executor/ob_task.cpp", "deserialize_tree", "t4", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_12, OB_ERR_UNEXPECTED, "executor/ob_task.cpp", "deserialize_tree", "t5", 1)
|
||||
EXCEPTION_TEST_1(et1_13, OB_ERROR, "executor/ob_task.cpp", "deserialize_tree", "t6", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_14, OB_ERROR, "executor/ob_task.cpp", "deserialize_tree", "t7", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_15, OB_ERROR, "executor/ob_distributed_job_control.cpp", "get_ready_jobs", "t1", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_16, OB_ERROR, "executor/ob_distributed_job_control.cpp", "get_ready_jobs", "t2", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_17, OB_ERR_UNEXPECTED, "executor/ob_distributed_job_executor.cpp", "execute_step", "t1", 1)
|
||||
EXCEPTION_TEST_1(et1_18, OB_NOT_INIT, "executor/ob_distributed_job_executor.cpp", "get_executable_tasks", "t1", 1)
|
||||
EXCEPTION_TEST_1(et1_19, OB_ERROR, "executor/ob_distributed_job_executor.cpp", "get_executable_tasks", "t2", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_20, OB_ERROR, "executor/ob_distributed_job_executor.cpp", "get_executable_tasks", "t3", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_21, OB_NOT_INIT, "executor/ob_distributed_task_executor.cpp", "execute", "t1", 1)
|
||||
EXCEPTION_TEST_1(et1_22, OB_NOT_INIT, "executor/ob_distributed_task_executor.cpp", "execute", "t2", 1)
|
||||
EXCEPTION_TEST_1(et1_23, OB_ERR_UNEXPECTED, "executor/ob_distributed_task_executor.cpp", "execute", "t3", 1)
|
||||
EXCEPTION_TEST_1(et1_24, OB_ERR_UNEXPECTED, "executor/ob_distributed_task_executor.cpp", "execute", "t4", 1)
|
||||
EXCEPTION_TEST_1(et1_25, OB_ERROR, "executor/ob_distributed_task_executor.cpp", "execute", "t5", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_26, OB_ERROR, "executor/ob_distributed_task_executor.cpp", "execute", "t6", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_27, OB_ERROR, "executor/ob_distributed_task_executor.cpp", "build_task", "t1", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_28, OB_NOT_INIT, "executor/ob_distributed_task_runner.cpp", "execute", "t1", 1)
|
||||
EXCEPTION_TEST_1(et1_29, OB_ERR_UNEXPECTED, "executor/ob_distributed_task_runner.cpp", "execute", "t2", 1)
|
||||
EXCEPTION_TEST_1(et1_30, OB_ERROR, "executor/ob_distributed_task_runner.cpp", "execute", "t3", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_31, OB_ERROR, "executor/ob_distributed_task_runner.cpp", "execute", "t4", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_32, OB_INVALID_ARGUMENT, "executor/ob_task_spliter.cpp", "init", "t1", 1)
|
||||
EXCEPTION_TEST_1(et1_33, OB_NOT_INIT, "executor/ob_identity_task_spliter.cpp", "get_next_task", "t1", 1)
|
||||
EXCEPTION_TEST_1(et1_34, OB_ALLOCATE_MEMORY_FAILED, "executor/ob_identity_task_spliter.cpp", "get_next_task", "t2", 1)
|
||||
EXCEPTION_TEST_1(et1_35, OB_NOT_INIT, "executor/ob_interm_task_spliter.cpp", "prepare", "t1", 1)
|
||||
EXCEPTION_TEST_1(et1_36, OB_ERR_UNEXPECTED, "executor/ob_interm_task_spliter.cpp", "prepare", "t2", 1)
|
||||
EXCEPTION_TEST_1(et1_37, OB_ALLOCATE_MEMORY_FAILED, "executor/ob_interm_task_spliter.cpp", "get_next_task", "t1", 1)
|
||||
EXCEPTION_TEST_1(et1_38, OB_NOT_INIT, "executor/ob_local_job_executor.cpp", "execute", "t1", 1)
|
||||
EXCEPTION_TEST_1(et1_39, OB_ERROR, "executor/ob_local_job_executor.cpp", "execute", "t2", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_40, OB_ERROR, "executor/ob_local_job_executor.cpp", "execute", "t3", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_41, OB_ERROR, "executor/ob_local_job_executor.cpp", "get_executable_task", "t1", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_42, OB_ERROR, "executor/ob_local_job_executor.cpp", "get_executable_task", "t2", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_43, OB_ERR_UNEXPECTED, "executor/ob_local_job_executor.cpp", "get_executable_task", "t3", 1)
|
||||
EXCEPTION_TEST_1(et1_44, OB_ERROR, "executor/ob_local_job_executor.cpp", "get_executable_task", "t4", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_45, OB_NOT_INIT, "executor/ob_job.cpp", "get_task_control", "t1", 1)
|
||||
EXCEPTION_TEST_1(et1_46, OB_ERROR, "executor/ob_job.cpp", "get_task_control", "t2", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_47, OB_ERROR, "executor/ob_job.cpp", "get_finished_task_locations", "t1", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_48, OB_ERR_UNEXPECTED, "executor/ob_job.cpp", "get_finished_task_locations", "t2", 1)
|
||||
//EXCEPTION_TEST_1(et1_49, OB_ERROR, "executor/ob_job.cpp", "get_finished_task_locations", "t3", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_50, OB_ERROR, "executor/ob_job.cpp", "get_finished_task_locations", "t4", OB_ERROR)
|
||||
//EXCEPTION_TEST_1(et1_51, OB_INVALID_ARGUMENT, "executor/ob_job.cpp", "update_job_state", "t1", 1)
|
||||
//EXCEPTION_TEST_1(et1_52, OB_INVALID_ARGUMENT, "executor/ob_job.cpp", "update_job_state", "t2", 1)
|
||||
//EXCEPTION_TEST_1(et1_53, OB_ERROR, "executor/ob_job.cpp", "update_job_state", "t3", OB_ERROR)
|
||||
//EXCEPTION_TEST_1(et1_54, OB_ERROR, "executor/ob_job.cpp", "update_job_state", "t4", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_55, OB_INVALID_ARGUMENT, "executor/ob_job.cpp", "add_child", "t1", 1)
|
||||
EXCEPTION_TEST_1(et1_56, OB_SIZE_OVERFLOW, "executor/ob_job.cpp", "add_child", "t2", 1)
|
||||
EXCEPTION_TEST_1(et1_57, OB_ERROR, "executor/ob_task_executor.cpp", "build_op_input", "t1", OB_ERROR)
|
||||
EXCEPTION_TEST_1(et1_58, OB_ERROR, "executor/ob_task_executor.cpp", "build_op_input", "t2", OB_ERROR)
|
||||
|
||||
EXCEPTION_TEST_2(et2_1, OB_ERROR,
|
||||
"executor/ob_distributed_task_runner.cpp", "execute", "t4", OB_ERROR,
|
||||
"executor/ob_distributed_job_executor.cpp", "kill_job", "t1", OB_ERROR)
|
||||
EXCEPTION_TEST_2(et2_2, OB_ERROR,
|
||||
"executor/ob_distributed_task_runner.cpp", "execute", "t4", OB_ERROR,
|
||||
"executor/ob_distributed_task_executor.cpp", "kill", "t1", 1)
|
||||
EXCEPTION_TEST_2(et2_3, OB_ERROR,
|
||||
"executor/ob_distributed_task_runner.cpp", "execute", "t4", OB_ERROR,
|
||||
"executor/ob_distributed_task_executor.cpp", "kill", "t2", 1)
|
||||
EXCEPTION_TEST_2(et2_4, OB_ERROR,
|
||||
"executor/ob_distributed_task_runner.cpp", "execute", "t4", OB_ERROR,
|
||||
"executor/ob_distributed_task_executor.cpp", "kill", "t3", 1)
|
||||
EXCEPTION_TEST_2(et2_5, OB_ERROR,
|
||||
"executor/ob_distributed_task_runner.cpp", "execute", "t4", OB_ERROR,
|
||||
"executor/ob_distributed_task_executor.cpp", "kill", "t4", OB_ERROR)
|
||||
EXCEPTION_TEST_2(et2_6, OB_ERROR,
|
||||
"executor/ob_distributed_task_runner.cpp", "execute", "t4", OB_ERROR,
|
||||
"executor/ob_distributed_task_executor.cpp", "kill", "t5", OB_ERROR)
|
||||
#endif
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
OB_LOGGER.set_log_level("INFO");
|
||||
oceanbase::common::ObLogger::get_logger().set_log_level("INFO");
|
||||
init_sql_factories();
|
||||
::testing::InitGoogleTest(&argc,argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
||||
434
unittest/sql/executor/test_executor.cpp
Normal file
434
unittest/sql/executor/test_executor.cpp
Normal file
@ -0,0 +1,434 @@
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
#include <stdarg.h>
|
||||
#include "sql/executor/ob_executor.h"
|
||||
#include "sql/executor/ob_distributed_scheduler.h"
|
||||
#include "ob_mock_utils.h"
|
||||
#include "sql/engine/table/ob_table_scan.h"
|
||||
#include "sql/engine/sort/ob_sort.h"
|
||||
#include "sql/executor/ob_root_transmit.h"
|
||||
#include "sql/executor/ob_distributed_transmit.h"
|
||||
#include "sql/executor/ob_direct_receive.h"
|
||||
#include "sql/executor/ob_fifo_receive.h"
|
||||
#include "sql/executor/ob_distributed_job_control.h"
|
||||
#include "sql/executor/ob_job_parser.h"
|
||||
#include "sql/executor/ob_task_spliter_factory.h"
|
||||
#include "sql/engine/ob_physical_plan_ctx.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/ob_sql_init.h"
|
||||
#include "lib/utility/ob_tracepoint.h"
|
||||
|
||||
using namespace oceanbase;
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
using namespace oceanbase::storage;
|
||||
using namespace oceanbase::share;
|
||||
|
||||
class ObExecutorTest : public ::testing::Test
|
||||
{
|
||||
public:
|
||||
|
||||
const static int64_t TEST_SPLIT_TASK_COUNT = 7;
|
||||
const static int64_t TEST_PARA_SERVER_COUNT = 2;
|
||||
const static int64_t TEST_SERVER_PARA_THREAD_COUNT = 3;
|
||||
|
||||
const static int64_t TEST_TABLE_ID = 1;
|
||||
const static int64_t TEST_INDEX_ID = 1;
|
||||
|
||||
const static int64_t TEST_LIMIT = 10;
|
||||
const static int64_t TEST_OFFSET = 0;
|
||||
|
||||
|
||||
ObExecutorTest();
|
||||
virtual ~ObExecutorTest();
|
||||
virtual void SetUp();
|
||||
virtual void TearDown();
|
||||
|
||||
ObMockSqlExecutorRpc rpc_;
|
||||
ObPhysicalPlan *local_phy_plan_;
|
||||
ObPhysicalPlan *distributed_phy_plan_;
|
||||
|
||||
int create_local_plan_tree(ObExecContext &ctx);
|
||||
int create_distributed_plan_tree(ObExecContext &ctx);
|
||||
private:
|
||||
// disallow copy
|
||||
ObExecutorTest(const ObExecutorTest &other);
|
||||
ObExecutorTest& operator=(const ObExecutorTest &other);
|
||||
private:
|
||||
// data members
|
||||
};
|
||||
ObExecutorTest::ObExecutorTest()
|
||||
{
|
||||
}
|
||||
|
||||
ObExecutorTest::~ObExecutorTest()
|
||||
{
|
||||
}
|
||||
|
||||
void ObExecutorTest::SetUp()
|
||||
{
|
||||
}
|
||||
|
||||
void ObExecutorTest::TearDown()
|
||||
{
|
||||
}
|
||||
|
||||
int ObExecutorTest::create_local_plan_tree(ObExecContext &ctx)
|
||||
{
|
||||
UNUSED(ctx);
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
local_phy_plan_ = ObPhysicalPlan::alloc();
|
||||
ObPhyOperator *cur_op = NULL;
|
||||
ObPhyOperator *tmp_op = NULL;
|
||||
int err_code = OB_SUCCESS;
|
||||
|
||||
ObSqlExpression *limit_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(local_phy_plan_, limit_expr));
|
||||
EXPECT_FALSE(NULL == limit_expr);
|
||||
ObPostExprItem limit_expr_item;
|
||||
limit_expr_item.set_int(TEST_LIMIT);
|
||||
limit_expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, limit_expr->add_expr_item(limit_expr_item));
|
||||
|
||||
ObSqlExpression *offset_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(local_phy_plan_, offset_expr));
|
||||
EXPECT_FALSE(NULL == offset_expr);
|
||||
ObPostExprItem offset_expr_item;
|
||||
offset_expr_item.set_int(TEST_OFFSET);
|
||||
offset_expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, offset_expr->add_expr_item(offset_expr_item));
|
||||
|
||||
|
||||
/*
|
||||
* calculate c0 % TEST_SPLIT_TASK_COUNT
|
||||
* */
|
||||
ObSqlExpression *hash_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(local_phy_plan_, hash_expr));
|
||||
EXPECT_FALSE(NULL == hash_expr);
|
||||
ObPostExprItem expr_item;
|
||||
expr_item.set_column(0);
|
||||
EXPECT_EQ(OB_SUCCESS, hash_expr->add_expr_item(expr_item));
|
||||
expr_item.set_int(TEST_SPLIT_TASK_COUNT);
|
||||
expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, hash_expr->add_expr_item(expr_item));
|
||||
expr_item.set_op("%", 2);
|
||||
EXPECT_EQ(OB_SUCCESS, hash_expr->add_expr_item(expr_item));
|
||||
|
||||
/*
|
||||
* calculate c0 % 1
|
||||
* */
|
||||
ObSqlExpression *iden_hash_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(local_phy_plan_, iden_hash_expr));
|
||||
EXPECT_FALSE(NULL == iden_hash_expr);
|
||||
ObPostExprItem iden_expr_item;
|
||||
iden_expr_item.set_column(0);
|
||||
EXPECT_EQ(OB_SUCCESS, iden_hash_expr->add_expr_item(iden_expr_item));
|
||||
iden_expr_item.set_int(1);
|
||||
iden_expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, iden_hash_expr->add_expr_item(iden_expr_item));
|
||||
iden_expr_item.set_op("%", 2);
|
||||
EXPECT_EQ(OB_SUCCESS, iden_hash_expr->add_expr_item(iden_expr_item));
|
||||
|
||||
uint64_t column_id1 = 16;
|
||||
ObColumnRefRawExpr ref_col(TEST_TABLE_ID, column_id1, T_REF_COLUMN);
|
||||
ObArray<ColumnItem> single_range_columns;
|
||||
ColumnItem col;
|
||||
col.column_id_ = column_id1;
|
||||
col.table_id_ = TEST_TABLE_ID;
|
||||
col.data_type_ = ObIntType;
|
||||
col.column_name_ = ObString::make_string("a");
|
||||
EXPECT_EQ(OB_SUCCESS, single_range_columns.push_back(col));
|
||||
ref_col.add_flag(IS_COLUMN);
|
||||
// 构造 (a = 5)
|
||||
ObObj value1;
|
||||
value1.set_int(5);
|
||||
ObConstRawExpr const_col1(value1, T_INT); //5
|
||||
const_col1.add_flag(IS_CONST);
|
||||
ObOpRawExpr condition1(&ref_col, &const_col1, T_OP_EQ); // a = 5构造完毕
|
||||
ObQueryRange *scan_query_range = OB_NEW(ObQueryRange, ObModIds::TEST);
|
||||
EXPECT_EQ(OB_SUCCESS, scan_query_range->preliminary_extract_query_range(single_range_columns, &condition1));
|
||||
|
||||
ASSERT_EQ(OB_SUCCESS, local_phy_plan_->alloc_operator_by_type(PHY_TABLE_SCAN, tmp_op));
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_table_id(TEST_TABLE_ID);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_ref_table_id(TEST_TABLE_ID);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_index_table_id(TEST_INDEX_ID);
|
||||
static_cast<ObTableScan*>(tmp_op)->add_range_column(column_id1);
|
||||
static_cast<ObTableScan*>(tmp_op)->add_output_column(column_id1);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_limit_offset(*limit_expr, *offset_expr);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_query_range(*scan_query_range);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
ASSERT_EQ(OB_SUCCESS, local_phy_plan_->alloc_operator_by_type(PHY_ROOT_TRANSMIT, tmp_op));
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_split_task_count(TEST_SPLIT_TASK_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_parallel_server_count(TEST_PARA_SERVER_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_server_parallel_thread_count(TEST_SERVER_PARA_THREAD_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_shuffle_func(hash_expr);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
local_phy_plan_->set_main_query(cur_op);
|
||||
local_phy_plan_->set_execute_type(OB_LOCAL_SINGLE_PARTITION_PLAN);
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExecutorTest::create_distributed_plan_tree(ObExecContext &ctx)
|
||||
{
|
||||
UNUSED(ctx);
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
distributed_phy_plan_ = ObPhysicalPlan::alloc();
|
||||
ObPhyOperator *cur_op = NULL;
|
||||
ObPhyOperator *tmp_op = NULL;
|
||||
int err_code = OB_SUCCESS;
|
||||
|
||||
ObSqlExpression *limit_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(distributed_phy_plan_, limit_expr));
|
||||
EXPECT_FALSE(NULL == limit_expr);
|
||||
ObPostExprItem limit_expr_item;
|
||||
limit_expr_item.set_int(TEST_LIMIT);
|
||||
limit_expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, limit_expr->add_expr_item(limit_expr_item));
|
||||
|
||||
ObSqlExpression *offset_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(distributed_phy_plan_, offset_expr));
|
||||
EXPECT_FALSE(NULL == offset_expr);
|
||||
ObPostExprItem offset_expr_item;
|
||||
offset_expr_item.set_int(TEST_OFFSET);
|
||||
offset_expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, offset_expr->add_expr_item(offset_expr_item));
|
||||
|
||||
|
||||
/*
|
||||
* calculate c0 % TEST_SPLIT_TASK_COUNT
|
||||
* */
|
||||
ObSqlExpression *hash_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(distributed_phy_plan_, hash_expr));
|
||||
EXPECT_FALSE(NULL == hash_expr);
|
||||
ObPostExprItem expr_item;
|
||||
expr_item.set_column(0);
|
||||
EXPECT_EQ(OB_SUCCESS, hash_expr->add_expr_item(expr_item));
|
||||
expr_item.set_int(TEST_SPLIT_TASK_COUNT);
|
||||
expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, hash_expr->add_expr_item(expr_item));
|
||||
expr_item.set_op("%", 2);
|
||||
EXPECT_EQ(OB_SUCCESS, hash_expr->add_expr_item(expr_item));
|
||||
|
||||
/*
|
||||
* calculate c0 % 1
|
||||
* */
|
||||
ObSqlExpression *iden_hash_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(distributed_phy_plan_, iden_hash_expr));
|
||||
EXPECT_FALSE(NULL == iden_hash_expr);
|
||||
ObPostExprItem iden_expr_item;
|
||||
iden_expr_item.set_column(0);
|
||||
EXPECT_EQ(OB_SUCCESS, iden_hash_expr->add_expr_item(iden_expr_item));
|
||||
iden_expr_item.set_int(1);
|
||||
iden_expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, iden_hash_expr->add_expr_item(iden_expr_item));
|
||||
iden_expr_item.set_op("%", 2);
|
||||
EXPECT_EQ(OB_SUCCESS, iden_hash_expr->add_expr_item(iden_expr_item));
|
||||
|
||||
uint64_t column_id1 = 16;
|
||||
ObColumnRefRawExpr ref_col(TEST_TABLE_ID, column_id1, T_REF_COLUMN);
|
||||
ObArray<ColumnItem> single_range_columns;
|
||||
ColumnItem col;
|
||||
col.column_id_ = column_id1;
|
||||
col.table_id_ = TEST_TABLE_ID;
|
||||
col.data_type_ = ObIntType;
|
||||
col.column_name_ = ObString::make_string("a");
|
||||
EXPECT_EQ(OB_SUCCESS, single_range_columns.push_back(col));
|
||||
ref_col.add_flag(IS_COLUMN);
|
||||
// 构造 (a = 5)
|
||||
ObObj value1;
|
||||
value1.set_int(5);
|
||||
ObConstRawExpr const_col1(value1, T_INT); //5
|
||||
const_col1.add_flag(IS_CONST);
|
||||
ObOpRawExpr condition1(&ref_col, &const_col1, T_OP_EQ); // a = 5构造完毕
|
||||
ObQueryRange *scan_query_range = OB_NEW(ObQueryRange, ObModIds::TEST);
|
||||
EXPECT_EQ(OB_SUCCESS, scan_query_range->preliminary_extract_query_range(single_range_columns, &condition1));
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObTableScan, PHY_TABLE_SCAN, distributed_phy_plan_, err_code);
|
||||
ASSERT_EQ(OB_SUCCESS, distributed_phy_plan_->alloc_operator_by_type(PHY_TABLE_SCAN, tmp_op));
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
cur_op = tmp_op;
|
||||
static_cast<ObTableScan*>(tmp_op)->set_table_id(TEST_TABLE_ID);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_ref_table_id(TEST_TABLE_ID);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_index_table_id(TEST_INDEX_ID);
|
||||
static_cast<ObTableScan*>(tmp_op)->add_range_column(column_id1);
|
||||
static_cast<ObTableScan*>(tmp_op)->add_output_column(column_id1);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_limit_offset(*limit_expr, *offset_expr);
|
||||
static_cast<ObTableScan*>(tmp_op)->set_query_range(*scan_query_range);
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObDistributedTransmit, PHY_DISTRIBUTED_TRANSMIT, distributed_phy_plan_, err_code);
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_split_task_count(TEST_SPLIT_TASK_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_parallel_server_count(TEST_PARA_SERVER_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_server_parallel_thread_count(TEST_SERVER_PARA_THREAD_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_shuffle_func(hash_expr);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObFifoReceive, PHY_FIFO_RECEIVE, distributed_phy_plan_, err_code);
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObDistributedTransmit, PHY_DISTRIBUTED_TRANSMIT, distributed_phy_plan_, err_code);
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::INTERM_SPLIT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_split_task_count(TEST_SPLIT_TASK_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_parallel_server_count(TEST_PARA_SERVER_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_server_parallel_thread_count(TEST_SERVER_PARA_THREAD_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_shuffle_func(iden_hash_expr);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObFifoReceive, PHY_FIFO_RECEIVE, distributed_phy_plan_, err_code);
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObRootTransmit, PHY_ROOT_TRANSMIT, distributed_phy_plan_, err_code);
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_split_task_count(TEST_SPLIT_TASK_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_parallel_server_count(TEST_PARA_SERVER_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_server_parallel_thread_count(TEST_SERVER_PARA_THREAD_COUNT);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_shuffle_func(hash_expr);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
distributed_phy_plan_->set_main_query(cur_op);
|
||||
distributed_phy_plan_->set_execute_type(OB_DISTRIBUTED_PLAN);
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
TEST_F(ObExecutorTest, local_executor_test)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
ObPhyTableLocationSEArray table_locs;
|
||||
ObPhyTableLocation table_loc;
|
||||
ObPartitionLocation partition_loc;
|
||||
partition_loc.set_table_id(TEST_TABLE_ID);
|
||||
partition_loc.set_partition_id(9);
|
||||
table_loc.set_table_id(TEST_TABLE_ID);
|
||||
ASSERT_EQ(OB_SUCCESS, table_loc.add_partition_location(partition_loc));
|
||||
ASSERT_EQ(OB_SUCCESS, table_locs.push_back(table_loc));
|
||||
|
||||
ObAddr server;
|
||||
server.set_ip_addr("127.0.0.1", 8888);
|
||||
ObArenaAllocator allocator(ObModIds::TEST);
|
||||
ObMockRemoteExecuteStreamHandle resp_handler(allocator);
|
||||
ObExecuteResult exe_result;
|
||||
ObExecContext exec_ctx;
|
||||
exec_ctx.init_phy_op(100);
|
||||
exec_ctx.create_physical_plan_ctx();
|
||||
/*
|
||||
ObPhysicalPlanCtx *plan_ctx = exec_ctx.get_physical_plan_ctx();
|
||||
plan_ctx->set_executor_rpc(rpc_);
|
||||
plan_ctx->set_task_response_handler(resp_handler);
|
||||
plan_ctx->set_server(server);
|
||||
plan_ctx->set_timeout_timestamp(::oceanbase::common::ObTimeUtility::current_time() + 2000L * 1000L);
|
||||
*/
|
||||
ObTaskExecutorCtx *task_exe_ctx = exec_ctx.get_task_executor_ctx();
|
||||
task_exe_ctx->set_partition_location_cache(&rpc_.partition_loc_cache_);
|
||||
task_exe_ctx->set_partition_service(&rpc_.partition_service_);
|
||||
task_exe_ctx->set_execute_result(&exe_result);
|
||||
task_exe_ctx->set_table_locations(table_locs);
|
||||
|
||||
ASSERT_EQ(OB_SUCCESS, create_local_plan_tree(exec_ctx));
|
||||
|
||||
ObExecutor ob_exe;
|
||||
ASSERT_EQ(OB_SUCCESS, ob_exe.execute_plan(exec_ctx, local_phy_plan_));
|
||||
ASSERT_EQ(OB_SUCCESS, exe_result.open(exec_ctx));
|
||||
const ObNewRow *tmp_row = NULL;
|
||||
while(OB_SUCCESS == (ret = exe_result.get_next_row(exec_ctx, tmp_row))) {
|
||||
SQL_EXE_LOG(INFO, "get a row", K(*tmp_row));
|
||||
}
|
||||
ASSERT_EQ(OB_ITER_END, ret);
|
||||
ASSERT_EQ(OB_SUCCESS, exe_result.close(exec_ctx));
|
||||
}
|
||||
|
||||
TEST_F(ObExecutorTest, distributed_executor_test)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
//启动模拟收包队列
|
||||
ObMockPacketQueueThread::get_instance()->start();
|
||||
|
||||
ObPhyTableLocationSEArray table_locs;
|
||||
ObPhyTableLocation table_loc;
|
||||
ObPartitionLocation partition_loc;
|
||||
partition_loc.set_table_id(TEST_TABLE_ID);
|
||||
partition_loc.set_partition_id(9);
|
||||
table_loc.set_table_id(TEST_TABLE_ID);
|
||||
ASSERT_EQ(OB_SUCCESS, table_loc.add_partition_location(partition_loc));
|
||||
ASSERT_EQ(OB_SUCCESS, table_locs.push_back(table_loc));
|
||||
|
||||
ObAddr server;
|
||||
server.set_ip_addr("127.0.0.1", 8888);
|
||||
ObArenaAllocator allocator(ObModIds::TEST);
|
||||
ObMockRemoteExecuteStreamHandle resp_handler(allocator);
|
||||
ObExecuteResult exe_result;
|
||||
ObExecContext exec_ctx;
|
||||
exec_ctx.init_phy_op(100);
|
||||
exec_ctx.create_physical_plan_ctx();
|
||||
ObPhysicalPlanCtx *plan_ctx = exec_ctx.get_physical_plan_ctx();
|
||||
//plan_ctx->set_task_response_handler(resp_handler);
|
||||
plan_ctx->set_server(server);
|
||||
plan_ctx->set_timeout_timestamp(::oceanbase::common::ObTimeUtility::current_time() + 2000L * 1000L);
|
||||
ObTaskExecutorCtx *task_exe_ctx = exec_ctx.get_task_executor_ctx();
|
||||
task_exe_ctx->set_partition_location_cache(&rpc_.partition_loc_cache_);
|
||||
task_exe_ctx->set_partition_service(&rpc_.partition_service_);
|
||||
task_exe_ctx->set_execute_result(&exe_result);
|
||||
task_exe_ctx->set_task_executor_rpc(&rpc_);
|
||||
task_exe_ctx->set_table_locations(table_locs);
|
||||
|
||||
ASSERT_EQ(OB_SUCCESS, create_distributed_plan_tree(exec_ctx));
|
||||
|
||||
ObExecutor ob_exe;
|
||||
ASSERT_EQ(OB_SUCCESS, ob_exe.execute_plan(exec_ctx, distributed_phy_plan_));
|
||||
ASSERT_EQ(OB_SUCCESS, exe_result.open(exec_ctx));
|
||||
const ObNewRow *tmp_row = NULL;
|
||||
while(OB_SUCCESS == (ret = exe_result.get_next_row(exec_ctx, tmp_row))) {
|
||||
SQL_EXE_LOG(INFO, "get a row", K(*tmp_row));
|
||||
}
|
||||
ASSERT_EQ(OB_ITER_END, ret);
|
||||
ASSERT_EQ(OB_SUCCESS, exe_result.close(exec_ctx));
|
||||
}
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
OB_LOGGER.set_log_level("INFO");
|
||||
oceanbase::common::ObLogger::get_logger().set_log_level("INFO");
|
||||
init_sql_factories();
|
||||
::testing::InitGoogleTest(&argc,argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
||||
267
unittest/sql/executor/test_interm_result.cpp
Normal file
267
unittest/sql/executor/test_interm_result.cpp
Normal file
@ -0,0 +1,267 @@
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include "common/row/ob_row_store.h"
|
||||
#include "lib/utility/ob_test_util.h"
|
||||
#include "common/row/ob_row.h"
|
||||
#include "sql/executor/ob_interm_result.h"
|
||||
#include "sql/executor/ob_interm_result_pool.h"
|
||||
#include "sql/executor/ob_interm_result_manager.h"
|
||||
#include "sql/executor/ob_task_event.h"
|
||||
#include "lib/allocator/ob_malloc.h"
|
||||
#include "lib/allocator/ob_mod_define.h"
|
||||
#include "lib/utility/ob_tracepoint.h"
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
using namespace oceanbase;
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
|
||||
class ObIntermResultTest : public ::testing::Test
|
||||
{
|
||||
public:
|
||||
static const int64_t SCANNER_NUM = 16;
|
||||
static const int64_t COL_NUM = 16;
|
||||
static const int64_t ROW_NUM = 100;
|
||||
static const int64_t BIG_ROW_NUM = 1024*1024*3;
|
||||
|
||||
ObIntermResultTest();
|
||||
virtual ~ObIntermResultTest();
|
||||
virtual void SetUp();
|
||||
virtual void TearDown();
|
||||
|
||||
void exception_test(int expected_ret, int64_t row_num);
|
||||
private:
|
||||
// disallow copy
|
||||
ObIntermResultTest(const ObIntermResultTest &other);
|
||||
ObIntermResultTest& operator=(const ObIntermResultTest &other);
|
||||
};
|
||||
|
||||
ObIntermResultTest::ObIntermResultTest()
|
||||
{
|
||||
}
|
||||
|
||||
ObIntermResultTest::~ObIntermResultTest()
|
||||
{
|
||||
}
|
||||
|
||||
void ObIntermResultTest::SetUp()
|
||||
{
|
||||
}
|
||||
|
||||
void ObIntermResultTest::TearDown()
|
||||
{
|
||||
}
|
||||
|
||||
void ObIntermResultTest::exception_test(int expected_ret, int64_t row_num)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(expected_ret);
|
||||
ObIntermResult tmp_ir;
|
||||
ObIntermResultPool* ir_pool = NULL;
|
||||
ObIntermResultItemPool* ir_item_pool = NULL;
|
||||
ObIntermResultManager* ir_manager = NULL;
|
||||
oceanbase::common::ObNewRow row;
|
||||
oceanbase::common::ObNewRow tmp_row;
|
||||
ObObj objs[COL_NUM];
|
||||
ObObj tmp_objs[COL_NUM];
|
||||
|
||||
ir_pool = ObIntermResultPool::get_instance();
|
||||
(void) ir_pool;
|
||||
ir_item_pool = ObIntermResultItemPool::get_instance();
|
||||
(void) ir_item_pool;
|
||||
ir_manager = ObIntermResultManager::get_instance();
|
||||
|
||||
row.count_ = COL_NUM;
|
||||
row.cells_ = objs;
|
||||
|
||||
tmp_row.count_ = COL_NUM;
|
||||
tmp_row.cells_ = tmp_objs;
|
||||
|
||||
ObIntermResult* ir = NULL;
|
||||
ObIntermResultIterator iter;
|
||||
int64_t expire_time = ::oceanbase::common::ObTimeUtility::current_time();
|
||||
|
||||
// add row
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < row_num; ++i) {
|
||||
for (int64_t j = 0; OB_SUCC(ret) && j < COL_NUM; ++j) {
|
||||
row.cells_[j].set_int(i*COL_NUM+j);
|
||||
} // end for
|
||||
ret = tmp_ir.add_row(OB_SYS_TENANT_ID, row);
|
||||
} // end for
|
||||
if (OB_SUCC(ret)) {
|
||||
ret = tmp_ir.complete_add_rows(OB_SYS_TENANT_ID);
|
||||
}
|
||||
|
||||
// add row
|
||||
if (OB_FAIL(ret)) {
|
||||
//empty
|
||||
} else if (OB_FAIL(ir_manager->alloc_result(ir))) {}
|
||||
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < row_num; ++i) {
|
||||
for (int64_t j = 0; OB_SUCC(ret) && j < COL_NUM; ++j) {
|
||||
row.cells_[j].set_int(i*COL_NUM+j);
|
||||
} // end for
|
||||
ret = ir->add_row(OB_SYS_TENANT_ID, row);
|
||||
} // end for
|
||||
if (OB_SUCC(ret)) {
|
||||
ret = ir->complete_add_rows(OB_SYS_TENANT_ID);
|
||||
}
|
||||
|
||||
if (OB_FAIL(ret)) {
|
||||
//empty
|
||||
} else if (OB_FAIL(ir_manager->free_result(ir))) {
|
||||
//empty
|
||||
} else if (OB_FAIL(ir_manager->alloc_result(ir))) {
|
||||
//empty
|
||||
}
|
||||
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < row_num; ++i) {
|
||||
for (int64_t j = 0; OB_SUCC(ret) && j < COL_NUM; ++j) {
|
||||
row.cells_[j].set_int(i*COL_NUM+j);
|
||||
} // end for
|
||||
OK(ir->add_row(OB_SYS_TENANT_ID, row));
|
||||
} // end for
|
||||
if (OB_SUCC(ret)) {
|
||||
ret = ir->complete_add_rows(OB_SYS_TENANT_ID);
|
||||
}
|
||||
|
||||
if (OB_FAIL(ret)) {
|
||||
//empty
|
||||
} else {
|
||||
ObScanner scanner;
|
||||
ObScanner::Iterator scanner_iter;
|
||||
ObIIntermResultItem *iir_item = NULL;
|
||||
ObIntermResultItem *ir_item = NULL;
|
||||
static int64_t id = 1;
|
||||
ObAddr server;
|
||||
server.set_ip_addr("127.0.0.1", 8888);
|
||||
ObSliceID slice_id;
|
||||
slice_id.set_server(server);
|
||||
slice_id.set_execution_id(id);
|
||||
slice_id.set_job_id(id);
|
||||
slice_id.set_task_id(id);
|
||||
slice_id.set_slice_id(id);
|
||||
id++;
|
||||
ObIntermResultInfo ir_info;
|
||||
ir_info.init(slice_id);
|
||||
if (OB_FAIL(ir_manager->add_result(ir_info, ir, expire_time))) {
|
||||
//empty
|
||||
} else if (OB_FAIL(ir_manager->get_result(ir_info, iter))) {
|
||||
//empty
|
||||
}
|
||||
|
||||
bool has_got_first_scanner = false;
|
||||
int64_t cur_row_num = 0;
|
||||
while (OB_SUCC(ret)) {
|
||||
bool should_get_next_item = false;
|
||||
if (!has_got_first_scanner) {
|
||||
should_get_next_item = true;
|
||||
} else {
|
||||
if (OB_FAIL(scanner_iter.get_next_row(tmp_row))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
should_get_next_item = true;
|
||||
}
|
||||
} else {
|
||||
for (int64_t j = 0; OB_SUCC(ret) && j < COL_NUM; ++j) {
|
||||
ASSERT_EQ(tmp_row.cells_[j].get_int(), cur_row_num*COL_NUM+j);
|
||||
}
|
||||
cur_row_num++;
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret) && should_get_next_item) {
|
||||
if (OB_FAIL(iter.get_next_interm_result_item(iir_item))) {
|
||||
} else {
|
||||
ASSERT_TRUE(NULL != iir_item);
|
||||
ir_item = static_cast<ObIntermResultItem *>(iir_item);
|
||||
scanner.reset();
|
||||
if (!scanner.is_inited() && OB_FAIL(scanner.init())) {
|
||||
} else if (OB_FAIL(ir_item->to_scanner(scanner))) {
|
||||
} else {
|
||||
scanner_iter = scanner.begin();
|
||||
has_got_first_scanner = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
ASSERT_EQ(cur_row_num, row_num);
|
||||
}
|
||||
|
||||
if (OB_FAIL(ret)) {
|
||||
//empty
|
||||
} else {
|
||||
ASSERT_EQ(OB_ITER_END, iter.get_next_interm_result_item(iir_item));
|
||||
ASSERT_EQ(OB_ITER_END, iter.get_next_interm_result_item(iir_item));
|
||||
if (OB_FAIL(ir_manager->delete_result(iter))) {
|
||||
//empty
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//OB_DELETE(ObIntermResultPool, ObModIds::OB_SQL_EXECUTOR, ir_pool);
|
||||
//OB_DELETE(ObScannerPool, ObModIds::OB_SQL_EXECUTOR, ir_item_pool);
|
||||
//OB_DELETE(ObIntermResultManager, ObModIds::OB_SQL_EXECUTOR, ir_manager);
|
||||
|
||||
//断点依赖路径正确,加上ccache以后,路径变成绝对路径了
|
||||
//ASSERT_EQ(expected_ret, ret);
|
||||
}
|
||||
|
||||
#define EXCEPTION_TEST(test_name, func, key, err, expect_ret, row_num) \
|
||||
TEST_F(ObIntermResultTest, test_name) \
|
||||
{\
|
||||
TP_SET_ERROR("executor/ob_interm_result.cpp", func, key, err); \
|
||||
exception_test(expect_ret, row_num); \
|
||||
TP_SET_ERROR("executor/ob_interm_result.cpp", func, key, NULL); \
|
||||
}\
|
||||
|
||||
|
||||
TEST_F(ObIntermResultTest, basic_test)
|
||||
{
|
||||
exception_test(OB_SUCCESS, BIG_ROW_NUM);
|
||||
}
|
||||
|
||||
EXCEPTION_TEST(et1, "ObIntermResult", "t1", 1, OB_SUCCESS, ROW_NUM);
|
||||
EXCEPTION_TEST(et2, "reset", "t1", 1, OB_SUCCESS, ROW_NUM);
|
||||
EXCEPTION_TEST(et3, "add_row", "t1", 1, OB_ERR_UNEXPECTED, ROW_NUM);
|
||||
EXCEPTION_TEST(et4, "add_row", "t2", 1, OB_ERR_UNEXPECTED, ROW_NUM);
|
||||
EXCEPTION_TEST(et5, "add_row", "t3", 1, OB_ERR_UNEXPECTED, ROW_NUM);
|
||||
EXCEPTION_TEST(et6, "add_row", "t4", OB_ERROR, OB_ERROR, ROW_NUM);
|
||||
EXCEPTION_TEST(et7, "add_row", "t5", OB_ERROR, OB_ERROR, ROW_NUM);
|
||||
EXCEPTION_TEST(et8, "add_row", "t6", OB_ERROR, OB_ERROR, ROW_NUM);
|
||||
EXCEPTION_TEST(et9, "set_interm_result", "t1", 1, OB_INVALID_ARGUMENT, ROW_NUM);
|
||||
EXCEPTION_TEST(et10, "set_interm_result", "t2", 1, OB_INVALID_ARGUMENT, ROW_NUM);
|
||||
EXCEPTION_TEST(et11, "get_interm_result_info", "t1", 1, OB_NOT_INIT, ROW_NUM);
|
||||
EXCEPTION_TEST(et12, "get_next_row", "t1", 1, OB_NOT_INIT, ROW_NUM);
|
||||
EXCEPTION_TEST(et13, "get_next_row", "t2", 1, OB_ERR_UNEXPECTED, ROW_NUM);
|
||||
EXCEPTION_TEST(et14, "try_inc_cnt", "t1", 1, OB_STATE_NOT_MATCH, ROW_NUM);
|
||||
EXCEPTION_TEST(et15, "try_dec_cnt", "t1", 1, OB_STATE_NOT_MATCH, ROW_NUM);
|
||||
EXCEPTION_TEST(et16, "try_begin_recycle", "t1", 1, OB_STATE_NOT_MATCH, ROW_NUM);
|
||||
EXCEPTION_TEST(et17, "try_end_recycle", "t1", 1, OB_SUCCESS, ROW_NUM);
|
||||
EXCEPTION_TEST(et18, "try_end_recycle", "t2", 1, OB_SUCCESS, ROW_NUM);
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
oceanbase::common::ObLogger::get_logger().set_log_level("INFO");
|
||||
ObIntermResultItemPool::build_instance();
|
||||
ObIntermResultPool::build_instance();
|
||||
ObIntermResultManager::build_instance();
|
||||
::testing::InitGoogleTest(&argc,argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
||||
92
unittest/sql/executor/test_interm_result_pool.cpp
Normal file
92
unittest/sql/executor/test_interm_result_pool.cpp
Normal file
@ -0,0 +1,92 @@
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include "sql/executor/ob_interm_result_pool.h"
|
||||
#include "sql/executor/ob_interm_result.h"
|
||||
#include "sql/executor/ob_scanner_pool.h"
|
||||
#include "sql/ob_sql_init.h"
|
||||
#include "lib/container/ob_array.h"
|
||||
#include <gtest/gtest.h>
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
|
||||
class ObIntermResultPoolTest : public ::testing::Test
|
||||
{
|
||||
public:
|
||||
ObIntermResultPoolTest();
|
||||
virtual ~ObIntermResultPoolTest();
|
||||
virtual void SetUp();
|
||||
virtual void TearDown();
|
||||
private:
|
||||
// disallow copy
|
||||
ObIntermResultPoolTest(const ObIntermResultPoolTest &other);
|
||||
ObIntermResultPoolTest& operator=(const ObIntermResultPoolTest &other);
|
||||
private:
|
||||
// data members
|
||||
};
|
||||
ObIntermResultPoolTest::ObIntermResultPoolTest()
|
||||
{
|
||||
}
|
||||
|
||||
ObIntermResultPoolTest::~ObIntermResultPoolTest()
|
||||
{
|
||||
}
|
||||
|
||||
void ObIntermResultPoolTest::SetUp()
|
||||
{
|
||||
}
|
||||
|
||||
void ObIntermResultPoolTest::TearDown()
|
||||
{
|
||||
}
|
||||
|
||||
TEST_F(ObIntermResultPoolTest, basic_test)
|
||||
{
|
||||
ObIntermResultPool tmp_ir_pool;
|
||||
UNUSED(tmp_ir_pool);
|
||||
ObIntermResultPool* ir_pool = ObIntermResultPool::get_instance();
|
||||
ObIntermResult* ir = NULL;
|
||||
ObIntermResult tmp_ir;
|
||||
tmp_ir.try_begin_recycle();
|
||||
|
||||
ASSERT_TRUE(NULL != ir_pool);
|
||||
ASSERT_EQ(OB_ERR_UNEXPECTED, ir_pool->free_interm_result(&tmp_ir));
|
||||
ASSERT_EQ(OB_INVALID_ARGUMENT, ir_pool->free_interm_result(ir));
|
||||
ASSERT_EQ(OB_SUCCESS, ir_pool->alloc_interm_result(ir));
|
||||
ASSERT_TRUE(NULL != ir);
|
||||
ASSERT_EQ(OB_SUCCESS, ir_pool->free_interm_result(ir));
|
||||
|
||||
ObArray<ObIntermResult*> tmp_arr;
|
||||
int64_t alloc_count = 100;
|
||||
int64_t free_count = 83;
|
||||
ASSERT_TRUE(alloc_count >= free_count);
|
||||
for (int64_t i = 0; i < alloc_count; ++i) {
|
||||
ASSERT_EQ(OB_SUCCESS, ir_pool->alloc_interm_result(ir));
|
||||
ASSERT_EQ(OB_SUCCESS, tmp_arr.push_back(ir));
|
||||
}
|
||||
int64_t left_count = ObScannerPool::get_capacity() - alloc_count;
|
||||
ASSERT_TRUE(left_count >= 0);
|
||||
for (int64_t i = 0; i < free_count; ++i) {
|
||||
ObIntermResult *tir = tmp_arr.at(i);
|
||||
ASSERT_EQ(OB_SUCCESS, ir_pool->free_interm_result(tir));
|
||||
}
|
||||
left_count = ObScannerPool::get_capacity() - alloc_count + free_count;
|
||||
ASSERT_TRUE(left_count >= 0);
|
||||
}
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
oceanbase::sql::init_sql_executor_singletons();
|
||||
oceanbase::common::ObLogger::get_logger().set_log_level("INFO");
|
||||
::testing::InitGoogleTest(&argc,argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
||||
385
unittest/sql/executor/test_job_control.cpp
Normal file
385
unittest/sql/executor/test_job_control.cpp
Normal file
@ -0,0 +1,385 @@
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include "ob_mock_utils.h"
|
||||
#include "sql/executor/ob_root_transmit.h"
|
||||
#include "sql/executor/ob_distributed_transmit.h"
|
||||
#include "sql/executor/ob_direct_receive.h"
|
||||
#include "sql/executor/ob_fifo_receive.h"
|
||||
#include "sql/executor/ob_distributed_job_control.h"
|
||||
#include "sql/executor/ob_job_parser.h"
|
||||
#include "sql/executor/ob_task_spliter_factory.h"
|
||||
#include "sql/engine/ob_physical_plan_ctx.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/executor/ob_local_job_control.h"
|
||||
#include "sql/executor/ob_remote_job_control.h"
|
||||
#include "sql/executor/ob_distributed_job_control.h"
|
||||
#include "sql/executor/ob_job.h"
|
||||
#include "sql/engine/table/ob_table_scan.h"
|
||||
#include "sql/ob_sql_init.h"
|
||||
#include "lib/utility/ob_tracepoint.h"
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
using namespace oceanbase;
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
using namespace oceanbase::storage;
|
||||
using namespace oceanbase::share;
|
||||
|
||||
class ObJobControlTest : public ::testing::Test
|
||||
{
|
||||
public:
|
||||
ObJobControlTest();
|
||||
virtual ~ObJobControlTest();
|
||||
virtual void SetUp();
|
||||
virtual void TearDown();
|
||||
|
||||
ObPhysicalPlan *phy_plan_;
|
||||
|
||||
int create_plan_tree(ObExecContext &ctx);
|
||||
void build_input_test(int exepected_ret, ObJobControl &jc);
|
||||
void exception_test(int expected_ret, ObJobControl &jc);
|
||||
void valid_test(ObJobControl &jc);
|
||||
private:
|
||||
// disallow copy
|
||||
ObJobControlTest(const ObJobControlTest &other);
|
||||
ObJobControlTest& operator=(const ObJobControlTest &other);
|
||||
private:
|
||||
// data members
|
||||
};
|
||||
ObJobControlTest::ObJobControlTest()
|
||||
{
|
||||
}
|
||||
|
||||
ObJobControlTest::~ObJobControlTest()
|
||||
{
|
||||
}
|
||||
|
||||
void ObJobControlTest::SetUp()
|
||||
{
|
||||
}
|
||||
|
||||
void ObJobControlTest::TearDown()
|
||||
{
|
||||
}
|
||||
|
||||
int ObJobControlTest::create_plan_tree(ObExecContext &ctx)
|
||||
{
|
||||
UNUSED(ctx);
|
||||
|
||||
const static int64_t TEST_PARA_DEGREE = 2;
|
||||
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
phy_plan_ = ObPhysicalPlan::alloc();
|
||||
ObPhyOperator *cur_op = NULL;
|
||||
ObPhyOperator *tmp_op = NULL;
|
||||
int err_code = OB_SUCCESS;
|
||||
|
||||
|
||||
/*
|
||||
* calculate c0 % TEST_PARA_DEGREE
|
||||
* */
|
||||
ObSqlExpression *hash_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(phy_plan_, hash_expr));
|
||||
EXPECT_FALSE(NULL == hash_expr);
|
||||
ObPostExprItem expr_item;
|
||||
expr_item.set_column(0);
|
||||
EXPECT_EQ(OB_SUCCESS, hash_expr->add_expr_item(expr_item));
|
||||
expr_item.set_int(TEST_PARA_DEGREE);
|
||||
expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, hash_expr->add_expr_item(expr_item));
|
||||
expr_item.set_op("%", 2);
|
||||
EXPECT_EQ(OB_SUCCESS, hash_expr->add_expr_item(expr_item));
|
||||
|
||||
/*
|
||||
* calculate c0 % 1
|
||||
* */
|
||||
ObSqlExpression *iden_hash_expr = NULL;
|
||||
EXPECT_EQ(OB_SUCCESS, ObSqlExpressionUtil::make_sql_expr(phy_plan_, iden_hash_expr));
|
||||
EXPECT_FALSE(NULL == iden_hash_expr);
|
||||
ObPostExprItem iden_expr_item;
|
||||
iden_expr_item.set_column(0);
|
||||
EXPECT_EQ(OB_SUCCESS, iden_hash_expr->add_expr_item(iden_expr_item));
|
||||
iden_expr_item.set_int(1);
|
||||
iden_expr_item.set_item_type(T_INT);
|
||||
EXPECT_EQ(OB_SUCCESS, iden_hash_expr->add_expr_item(iden_expr_item));
|
||||
iden_expr_item.set_op("%", 2);
|
||||
EXPECT_EQ(OB_SUCCESS, iden_hash_expr->add_expr_item(iden_expr_item));
|
||||
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObTableScan, PHY_TABLE_SCAN, phy_plan_, err_code);
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObDistributedTransmit, PHY_DISTRIBUTED_TRANSMIT, phy_plan_, err_code);
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
//static_cast<ObTransmit*>(tmp_op)->set_parallel_degree(TEST_PARA_DEGREE);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_shuffle_func(hash_expr);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObFifoReceive, PHY_FIFO_RECEIVE, phy_plan_, err_code);
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObDistributedTransmit, PHY_DISTRIBUTED_TRANSMIT, phy_plan_, err_code);
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::INTERM_SPLIT);
|
||||
//static_cast<ObTransmit*>(tmp_op)->set_parallel_degree(TEST_PARA_DEGREE);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_shuffle_func(iden_hash_expr);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObFifoReceive, PHY_FIFO_RECEIVE, phy_plan_, err_code);
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObRootTransmit, PHY_ROOT_TRANSMIT, phy_plan_, err_code);
|
||||
tmp_op->set_column_count(TEST_MOCK_COL_NUM);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
//static_cast<ObTransmit*>(tmp_op)->set_parallel_degree(TEST_PARA_DEGREE);
|
||||
static_cast<ObTransmit*>(tmp_op)->set_shuffle_func(hash_expr);
|
||||
cur_op = tmp_op;
|
||||
SQL_EXE_LOG(INFO, "op info", "op_id", cur_op->get_id(), "op_type", cur_op->get_type());
|
||||
|
||||
phy_plan_->set_main_query(cur_op);
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObJobControlTest::build_input_test(int expected_ret, ObJobControl &jc) {
|
||||
int ret = OB_SUCCESS;
|
||||
ObAddrsProviderFactory spf;
|
||||
ObTaskSpliterFactory tsf;
|
||||
ObJobParser parser;
|
||||
ObExecContext exec_ctx;
|
||||
exec_ctx.init_phy_op(100);
|
||||
exec_ctx.create_physical_plan_ctx();
|
||||
|
||||
create_plan_tree(exec_ctx);
|
||||
if (OB_FAIL(jc.build_jobs_ctx(exec_ctx))) {
|
||||
//empty
|
||||
} else if (OB_FAIL(parser.parse_job(exec_ctx, phy_plan_, 1, tsf, spf, jc))) {
|
||||
//empty
|
||||
}
|
||||
ASSERT_EQ(expected_ret, ret);
|
||||
}
|
||||
|
||||
void ObJobControlTest::exception_test(int expected_ret, ObJobControl &jc)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObJob *job1 = NULL;
|
||||
ObJob *job2 = NULL;
|
||||
ObJob *job3 = NULL;
|
||||
ObJob *tmp_job = NULL;
|
||||
ObArray<ObJob *> jobs;
|
||||
ObArray<ObJob *> jobs1;
|
||||
ObArray<ObJob *> jobs2;
|
||||
ObArenaAllocator allocator(ObModIds::OB_SQL_EXEC_CONTEXT);
|
||||
int64_t query_id = OB_INVALID_ID;
|
||||
|
||||
if (OB_FAIL(jc.create_job(allocator, query_id, job1))) {
|
||||
//empty
|
||||
} else if (OB_FAIL(jc.create_job(allocator, query_id, job2))) {
|
||||
//empty
|
||||
} else if (OB_FAIL(jc.create_job(allocator, query_id, job3))) {
|
||||
//empty
|
||||
} else {
|
||||
job1->set_priority(1);
|
||||
job2->set_priority(2);
|
||||
job3->set_priority(3);
|
||||
if (OB_FAIL(jc.add_job(job1))) {
|
||||
//empty
|
||||
} else if (OB_FAIL(jc.add_job(job2))) {
|
||||
//empty
|
||||
} else if (OB_FAIL(jc.add_job(job3))) {
|
||||
//empty
|
||||
} else if (OB_FAIL(jc.find_job(job1->get_job_id(), tmp_job))) {
|
||||
//empty
|
||||
} else if (OB_FAIL(jc.get_all_jobs(jobs))) {
|
||||
//empty
|
||||
} else {
|
||||
OB_ASSERT(3 == jobs.count());
|
||||
OB_ASSERT(1 == jobs.at(0)->get_priority());
|
||||
OB_ASSERT(2 == jobs.at(1)->get_priority());
|
||||
OB_ASSERT(3 == jobs.at(2)->get_priority());
|
||||
if (OB_FAIL(jc.arrange_jobs())) {
|
||||
//empty
|
||||
} else if (OB_FAIL(jc.get_all_jobs(jobs1))) {
|
||||
//empty
|
||||
} else {
|
||||
bool b1 = true;
|
||||
bool b2 = true;
|
||||
OB_ASSERT(3 == jobs1.count());
|
||||
OB_ASSERT(3 == jobs1.at(0)->get_priority());
|
||||
OB_ASSERT(2 == jobs1.at(1)->get_priority());
|
||||
OB_ASSERT(1 == jobs1.at(2)->get_priority());
|
||||
if(OB_FAIL(jc.all_jobs_finished(b1))) {
|
||||
//empty
|
||||
} else if (OB_FAIL(jc.all_jobs_finished_except_root_job(b2))) {
|
||||
//empty
|
||||
} else {
|
||||
OB_ASSERT(false == b1);
|
||||
OB_ASSERT(false == b2);
|
||||
if (OB_FAIL(jc.get_running_jobs(jobs2))) {
|
||||
//empty
|
||||
} else {
|
||||
OB_ASSERT(0 == jobs2.count());
|
||||
jobs1.at(0)->set_state(OB_JOB_STATE_RUNNING);
|
||||
OB_ASSERT(OB_SUCCESS == jc.all_jobs_finished(b1));
|
||||
OB_ASSERT(OB_SUCCESS == jc.all_jobs_finished_except_root_job(b2));
|
||||
OB_ASSERT(false == b1);
|
||||
OB_ASSERT(false == b2);
|
||||
jobs2.reset();
|
||||
if (OB_FAIL(jc.get_running_jobs(jobs2))) {
|
||||
//empty
|
||||
} else {
|
||||
OB_ASSERT(1 == jobs2.count());
|
||||
OB_ASSERT(3 == jobs2.at(0)->get_priority());
|
||||
jobs1.at(1)->set_state(OB_JOB_STATE_RUNNING);
|
||||
OB_ASSERT(OB_SUCCESS == jc.all_jobs_finished(b1));
|
||||
OB_ASSERT(OB_SUCCESS == jc.all_jobs_finished_except_root_job(b2));
|
||||
OB_ASSERT(false == b1);
|
||||
OB_ASSERT(false == b2);
|
||||
jobs2.reset();
|
||||
if (OB_FAIL(jc.get_running_jobs(jobs2))) {
|
||||
//empty
|
||||
} else {
|
||||
OB_ASSERT(2 == jobs2.count());
|
||||
OB_ASSERT(3 == jobs2.at(0)->get_priority());
|
||||
OB_ASSERT(2 == jobs2.at(1)->get_priority());
|
||||
jobs1.at(2)->set_state(OB_JOB_STATE_RUNNING);
|
||||
OB_ASSERT(OB_SUCCESS == jc.all_jobs_finished(b1));
|
||||
OB_ASSERT(OB_SUCCESS == jc.all_jobs_finished_except_root_job(b2));
|
||||
OB_ASSERT(false == b1);
|
||||
OB_ASSERT(false == b2);
|
||||
jobs2.reset();
|
||||
if (OB_FAIL(jc.get_running_jobs(jobs2))) {
|
||||
//empty
|
||||
} else {
|
||||
OB_ASSERT(3 == jobs2.count());
|
||||
OB_ASSERT(3 == jobs2.at(0)->get_priority());
|
||||
OB_ASSERT(2 == jobs2.at(1)->get_priority());
|
||||
OB_ASSERT(1 == jobs2.at(2)->get_priority());
|
||||
jobs1.at(0)->set_state(OB_JOB_STATE_FINISHED);
|
||||
jobs1.at(1)->set_state(OB_JOB_STATE_FINISHED);
|
||||
OB_ASSERT(OB_SUCCESS == jc.all_jobs_finished(b1));
|
||||
OB_ASSERT(OB_SUCCESS == jc.all_jobs_finished_except_root_job(b2));
|
||||
OB_ASSERT(false == b1);
|
||||
OB_ASSERT(true == b2);
|
||||
jobs1.at(2)->set_state(OB_JOB_STATE_FINISHED);
|
||||
OB_ASSERT(OB_SUCCESS == jc.all_jobs_finished(b1));
|
||||
OB_ASSERT(OB_SUCCESS == jc.all_jobs_finished_except_root_job(b2));
|
||||
OB_ASSERT(true == b1);
|
||||
OB_ASSERT(true == b2);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
ASSERT_EQ(expected_ret, ret);
|
||||
}
|
||||
|
||||
void ObJobControlTest::valid_test(ObJobControl &jc)
|
||||
{
|
||||
ObJob *job = NULL;
|
||||
ObJob *job2 = NULL;
|
||||
bool b = true;
|
||||
ObArenaAllocator allocator(ObModIds::OB_SQL_EXEC_CONTEXT);
|
||||
int64_t query_id = OB_INVALID_ID;
|
||||
OB_ASSERT(OB_SUCCESS == jc.create_job(allocator, query_id, job));
|
||||
job->set_root_job();
|
||||
OB_ASSERT(true == job->is_root_job());
|
||||
OB_ASSERT(OB_ENTRY_NOT_EXIST == jc.find_job(100, job2));
|
||||
OB_ASSERT(NULL == job2);
|
||||
OB_ASSERT(OB_ENTRY_NOT_EXIST == jc.find_job(job->get_job_id(), job2));
|
||||
OB_ASSERT(OB_ERR_UNEXPECTED == jc.all_jobs_finished_except_root_job(b));
|
||||
OB_ASSERT(OB_SUCCESS == jc.add_job(job));
|
||||
OB_ASSERT(OB_SUCCESS == jc.find_job(job->get_job_id(), job2));
|
||||
OB_ASSERT(job2->get_job_id() == job->get_job_id());
|
||||
OB_ASSERT(true == job2->is_root_job());
|
||||
OB_ASSERT(OB_SUCCESS == jc.all_jobs_finished_except_root_job(b));
|
||||
}
|
||||
|
||||
TEST_F(ObJobControlTest, valid_test)
|
||||
{
|
||||
ObLocalJobControl local_jc;
|
||||
ObLocalJobControl remote_jc;
|
||||
ObLocalJobControl distributed_jc;
|
||||
valid_test(local_jc);
|
||||
valid_test(remote_jc);
|
||||
valid_test(distributed_jc);
|
||||
}
|
||||
|
||||
TEST_F(ObJobControlTest, build_input_test)
|
||||
{
|
||||
ObLocalJobControl local_jc;
|
||||
ObLocalJobControl remote_jc;
|
||||
ObLocalJobControl distributed_jc;
|
||||
build_input_test(OB_SUCCESS, local_jc);
|
||||
build_input_test(OB_SUCCESS, remote_jc);
|
||||
build_input_test(OB_SUCCESS, distributed_jc);
|
||||
}
|
||||
|
||||
TEST_F(ObJobControlTest, basic_test)
|
||||
{
|
||||
ObLocalJobControl local_jc;
|
||||
ObLocalJobControl remote_jc;
|
||||
ObLocalJobControl distributed_jc;
|
||||
exception_test(OB_SUCCESS, local_jc);
|
||||
exception_test(OB_SUCCESS, remote_jc);
|
||||
exception_test(OB_SUCCESS, distributed_jc);
|
||||
}
|
||||
|
||||
#define EXCEPTION_TEST(test_func, test_name, func, key, err, expect_ret) \
|
||||
TEST_F(ObJobControlTest, test_name) \
|
||||
{\
|
||||
ObLocalJobControl local_jc; \
|
||||
ObLocalJobControl remote_jc; \
|
||||
ObLocalJobControl distributed_jc; \
|
||||
TP_SET_ERROR("executor/ob_job_control.cpp", func, key, err); \
|
||||
test_func(expect_ret, local_jc); \
|
||||
test_func(expect_ret, remote_jc); \
|
||||
test_func(expect_ret, distributed_jc); \
|
||||
TP_SET_ERROR("executor/ob_job_control.cpp", func, key, NULL); \
|
||||
}\
|
||||
|
||||
EXCEPTION_TEST(exception_test, et1, "all_jobs_finished", "t1", 1, OB_ERR_UNEXPECTED);
|
||||
EXCEPTION_TEST(exception_test, et2, "all_jobs_finished_except_root_job", "t1", 1, OB_ERR_UNEXPECTED);
|
||||
EXCEPTION_TEST(exception_test, et3, "find_job", "t1", 1, OB_ERR_UNEXPECTED);
|
||||
EXCEPTION_TEST(exception_test, et4, "get_running_jobs", "t1", 1, OB_ERR_UNEXPECTED);
|
||||
EXCEPTION_TEST(exception_test, et5, "get_all_jobs", "t1", 1, OB_ERR_UNEXPECTED);
|
||||
EXCEPTION_TEST(exception_test, et6, "create_job", "t1", 1, OB_ALLOCATE_MEMORY_FAILED);
|
||||
EXCEPTION_TEST(exception_test, et7, "create_job", "t2", 1, OB_ERR_UNEXPECTED);
|
||||
//EXCEPTION_TEST(build_input_test, et8, "build_job_ctx", "t1", 1, OB_ERR_UNEXPECTED);
|
||||
EXCEPTION_TEST(build_input_test, et9, "build_job_ctx", "t2", 1, OB_ERR_UNEXPECTED);
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
init_sql_factories();
|
||||
oceanbase::common::ObLogger::get_logger().set_log_level("INFO");
|
||||
::testing::InitGoogleTest(&argc,argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
||||
172
unittest/sql/executor/test_job_id.cpp
Normal file
172
unittest/sql/executor/test_job_id.cpp
Normal file
@ -0,0 +1,172 @@
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
#include "sql/executor/ob_job_id.h"
|
||||
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
|
||||
class ObJobIDTest : public ::testing::Test
|
||||
{
|
||||
public:
|
||||
ObJobIDTest();
|
||||
virtual ~ObJobIDTest();
|
||||
virtual void SetUp();
|
||||
virtual void TearDown();
|
||||
private:
|
||||
// disallow copy
|
||||
ObJobIDTest(const ObJobIDTest &other);
|
||||
ObJobIDTest& operator=(const ObJobIDTest &other);
|
||||
private:
|
||||
// data members
|
||||
};
|
||||
ObJobIDTest::ObJobIDTest()
|
||||
{
|
||||
}
|
||||
|
||||
ObJobIDTest::~ObJobIDTest()
|
||||
{
|
||||
}
|
||||
|
||||
void ObJobIDTest::SetUp()
|
||||
{
|
||||
}
|
||||
|
||||
void ObJobIDTest::TearDown()
|
||||
{
|
||||
}
|
||||
|
||||
TEST_F(ObJobIDTest, basic_test)
|
||||
{
|
||||
const static uint64_t valid_id_a = 1033;
|
||||
const static uint64_t valid_id_b = 42233;
|
||||
ObJobID job_id;
|
||||
|
||||
// initial state
|
||||
ASSERT_EQ(OB_INVALID_ID, job_id.get_execution_id());
|
||||
ASSERT_EQ(OB_INVALID_ID, job_id.get_job_id());
|
||||
|
||||
// state change
|
||||
job_id.set_execution_id(valid_id_a);
|
||||
job_id.set_job_id(valid_id_b);
|
||||
ASSERT_EQ(valid_id_a, job_id.get_execution_id());
|
||||
ASSERT_EQ(valid_id_b, job_id.get_job_id());
|
||||
}
|
||||
|
||||
TEST_F(ObJobIDTest, equal_test)
|
||||
{
|
||||
const static uint64_t valid_id_a = 1033;
|
||||
const static uint64_t valid_id_b = 42233;
|
||||
ObJobID job_id;
|
||||
ObJobID job_id2;
|
||||
|
||||
// consist initial state
|
||||
ASSERT_EQ(true, job_id.equal(job_id2));
|
||||
ASSERT_EQ(true, job_id2.equal(job_id));
|
||||
|
||||
// state change
|
||||
job_id.set_execution_id(valid_id_a);
|
||||
job_id.set_job_id(valid_id_b);
|
||||
job_id2.set_execution_id(valid_id_a);
|
||||
job_id2.set_job_id(valid_id_b);
|
||||
ASSERT_EQ(true, job_id.equal(job_id2));
|
||||
ASSERT_EQ(true, job_id2.equal(job_id));
|
||||
|
||||
// state change
|
||||
job_id.set_execution_id(valid_id_a);
|
||||
job_id.set_job_id(valid_id_b);
|
||||
job_id2.set_execution_id(valid_id_a);
|
||||
job_id2.set_job_id(valid_id_a);
|
||||
EXPECT_TRUE(false == job_id.equal(job_id2));
|
||||
EXPECT_TRUE(false == job_id2.equal(job_id));
|
||||
|
||||
job_id.reset();
|
||||
job_id2.reset();
|
||||
EXPECT_TRUE(false == job_id.is_valid());
|
||||
EXPECT_TRUE(false == job_id2.is_valid());
|
||||
EXPECT_TRUE(true == job_id.equal(job_id2));
|
||||
}
|
||||
|
||||
|
||||
TEST_F(ObJobIDTest, valid_test)
|
||||
{
|
||||
const static uint64_t valid_id = 1;
|
||||
ObJobID job_id;
|
||||
job_id.set_execution_id(OB_INVALID_ID);
|
||||
job_id.set_job_id(OB_INVALID_ID);
|
||||
EXPECT_TRUE(false == job_id.is_valid());
|
||||
|
||||
job_id.set_server(ObAddr(ObAddr::IPV4, "127.0.0.1", 8888));
|
||||
job_id.set_execution_id(OB_INVALID_ID);
|
||||
job_id.set_job_id(OB_INVALID_ID);
|
||||
EXPECT_TRUE(false == job_id.is_valid());
|
||||
|
||||
job_id.set_execution_id(OB_INVALID_ID);
|
||||
job_id.set_job_id(valid_id);
|
||||
EXPECT_TRUE(false == job_id.is_valid());
|
||||
|
||||
|
||||
job_id.set_execution_id(valid_id);
|
||||
job_id.set_job_id(OB_INVALID_ID);
|
||||
EXPECT_TRUE(false == job_id.is_valid());
|
||||
|
||||
|
||||
job_id.set_execution_id(valid_id);
|
||||
job_id.set_job_id(valid_id);
|
||||
EXPECT_TRUE(true == job_id.is_valid());
|
||||
|
||||
}
|
||||
|
||||
TEST_F(ObJobIDTest, serialize_test)
|
||||
{
|
||||
ObJobID job_id;
|
||||
ObJobID job_id2;
|
||||
|
||||
job_id.set_server(ObAddr(ObAddr::IPV4, "127.0.0.1", 8888));
|
||||
job_id.set_execution_id(OB_INVALID_ID);
|
||||
job_id.set_job_id(OB_INVALID_ID);
|
||||
|
||||
// invalid val deserialize
|
||||
char buf[1024];
|
||||
int64_t buf_len = 1024;
|
||||
int64_t pos = 0;
|
||||
int64_t pos2 = 0;
|
||||
int ret = job_id.serialize(buf, buf_len, pos);
|
||||
EXPECT_TRUE(OB_SUCC(ret));
|
||||
job_id2.deserialize(buf, pos, pos2);
|
||||
EXPECT_TRUE(false == job_id.is_valid());
|
||||
EXPECT_TRUE(false == job_id2.is_valid());
|
||||
EXPECT_TRUE(true == job_id.equal(job_id2));
|
||||
|
||||
// valid val deserialize
|
||||
uint64_t valid_id_a = 1012;
|
||||
uint64_t valid_id_b = 13012;
|
||||
job_id.set_execution_id(valid_id_a);
|
||||
job_id.set_job_id(valid_id_b);
|
||||
ret = job_id.serialize(buf, buf_len, pos);
|
||||
EXPECT_TRUE(OB_SUCC(ret));
|
||||
job_id2.deserialize(buf, pos, pos2);
|
||||
EXPECT_TRUE(true == job_id.is_valid());
|
||||
EXPECT_TRUE(true == job_id2.is_valid());
|
||||
EXPECT_TRUE(true == job_id.equal(job_id2));
|
||||
|
||||
SQL_EXE_LOG(INFO, "info", K(job_id));
|
||||
|
||||
}
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
oceanbase::common::ObLogger::get_logger().set_log_level("INFO");
|
||||
::testing::InitGoogleTest(&argc,argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
||||
420
unittest/sql/executor/test_job_parser.cpp
Normal file
420
unittest/sql/executor/test_job_parser.cpp
Normal file
@ -0,0 +1,420 @@
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
#include "ob_mock_utils.h"
|
||||
#include "lib/allocator/page_arena.h"
|
||||
#include "sql/ob_sql_init.h"
|
||||
//#include "sql/engine/basic/ob_project.h"
|
||||
#include "sql/engine/sort/ob_sort.h"
|
||||
//#include "sql/engine/join/ob_merge_join.h"
|
||||
//#include "sql/engine/dml/ob_when_filter.h"
|
||||
#include "sql/executor/ob_root_transmit.h"
|
||||
#include "sql/executor/ob_direct_receive.h"
|
||||
#include "sql/engine/table/ob_table_scan.h"
|
||||
#include "sql/executor/ob_distributed_job_control.h"
|
||||
#include "sql/executor/ob_job_parser.h"
|
||||
#include "sql/executor/ob_task_spliter_factory.h"
|
||||
|
||||
using namespace oceanbase;
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
|
||||
class ObJobParserTest : public ::testing::Test
|
||||
{
|
||||
public:
|
||||
ObJobParserTest();
|
||||
virtual ~ObJobParserTest();
|
||||
virtual void SetUp();
|
||||
virtual void TearDown();
|
||||
static void print_phy_op_tree(char *buf, const int64_t buf_len, int64_t& pos, ObPhyOperator *op);
|
||||
static void print_sub_phy_op_tree(char *buf, const int64_t buf_len, int64_t& pos, ObPhyOperator *op);
|
||||
static void print_job_tree(char *buf, const int64_t buf_len, int64_t& pos, ObJobControl *jc);
|
||||
private:
|
||||
// disallow copy
|
||||
ObJobParserTest(const ObJobParserTest &other);
|
||||
ObJobParserTest& operator=(const ObJobParserTest &ohter);
|
||||
private:
|
||||
static void print_op_tree(char *buf, const int64_t buf_len, int64_t& pos, ObPhyOperator *op, int32_t level);
|
||||
static void print_sub_op_tree(char *buf, const int64_t buf_len, int64_t& pos, ObPhyOperator *op, int32_t level);
|
||||
};
|
||||
|
||||
|
||||
ObJobParserTest::ObJobParserTest()
|
||||
{
|
||||
}
|
||||
|
||||
ObJobParserTest::~ObJobParserTest()
|
||||
{
|
||||
}
|
||||
|
||||
void ObJobParserTest::SetUp()
|
||||
{
|
||||
}
|
||||
|
||||
void ObJobParserTest::TearDown()
|
||||
{
|
||||
}
|
||||
|
||||
void ObJobParserTest::print_phy_op_tree(char *buf, const int64_t buf_len, int64_t& pos, ObPhyOperator *op)
|
||||
{
|
||||
ObJobParserTest::print_op_tree(buf, buf_len, pos, op, 0);
|
||||
buf[pos] = '\0';
|
||||
}
|
||||
|
||||
void ObJobParserTest::print_op_tree(char *buf, const int64_t buf_len, int64_t& pos, ObPhyOperator *op, int32_t level)
|
||||
{
|
||||
const char* op_name = NULL;
|
||||
if (NULL != op)
|
||||
{
|
||||
op_name = ob_phy_operator_type_str(op->get_type());
|
||||
}
|
||||
for (int32_t i = 0; i < level; i++)
|
||||
{
|
||||
::oceanbase::common::databuff_printf(buf, buf_len, pos, " ");
|
||||
}
|
||||
::oceanbase::common::databuff_printf(buf, buf_len, pos, "|- %s\n", op_name);
|
||||
if (NULL != op)
|
||||
{
|
||||
for (int32_t i = 0; i < op->get_child_num(); i++)
|
||||
{
|
||||
ObJobParserTest::print_op_tree(buf, buf_len, pos, op->get_child(i), level + 1);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void ObJobParserTest::print_job_tree(char *buf, const int64_t buf_len, int64_t& pos, ObJobControl *jc)
|
||||
{
|
||||
::oceanbase::common::databuff_printf(buf, buf_len, pos, "total jobs: %ld\n", jc->get_job_count());
|
||||
ObArray<ObJob *> jobs;
|
||||
jc->get_all_jobs(jobs);
|
||||
for (int64_t i = 0; i < jobs.count(); ++i)
|
||||
{
|
||||
ObJob *job = jobs.at(i);
|
||||
::oceanbase::common::databuff_printf(buf, buf_len, pos, "job_id=%ld, priority=%ld:\n", i, job->get_priority());
|
||||
ObJobParserTest::print_sub_phy_op_tree(buf, buf_len, pos, job->get_root_op());
|
||||
}
|
||||
buf[pos] = '\0';
|
||||
}
|
||||
|
||||
void ObJobParserTest::print_sub_phy_op_tree(char *buf, const int64_t buf_len, int64_t& pos, ObPhyOperator *op)
|
||||
{
|
||||
ObJobParserTest::print_sub_op_tree(buf, buf_len, pos, op, 0);
|
||||
buf[pos] = '\0';
|
||||
}
|
||||
|
||||
void ObJobParserTest::print_sub_op_tree(char *buf, const int64_t buf_len, int64_t& pos, ObPhyOperator *op, int32_t level)
|
||||
{
|
||||
const char* op_name = NULL;
|
||||
if (NULL != op)
|
||||
{
|
||||
op_name = ob_phy_operator_type_str(op->get_type());
|
||||
}
|
||||
for (int32_t i = 0; i < level; i++)
|
||||
{
|
||||
::oceanbase::common::databuff_printf(buf, buf_len, pos, " ");
|
||||
}
|
||||
::oceanbase::common::databuff_printf(buf, buf_len, pos, "|- %s\n", op_name);
|
||||
if (NULL != op)
|
||||
{
|
||||
for (int32_t i = 0; i < op->get_child_num(); i++)
|
||||
{
|
||||
ObTransmit* trans_op = dynamic_cast<ObTransmit*>(op->get_child(i));
|
||||
if (NULL == trans_op)
|
||||
{
|
||||
ObJobParserTest::print_sub_op_tree(buf, buf_len, pos, op->get_child(i), level + 1);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(ObJobParserTest, basic_test)
|
||||
{
|
||||
ObPhysicalPlan *physical_plan = ObPhysicalPlan::alloc();
|
||||
ObPhyOperator *cur_op = NULL;
|
||||
ObPhyOperator *tmp_op = NULL;
|
||||
int err_code = OB_SUCCESS;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObTableScan, PHY_TABLE_SCAN, physical_plan, err_code);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObSort, PHY_SORT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObSort, PHY_SORT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObRootTransmit, PHY_ROOT_TRANSMIT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObDirectReceive, PHY_DIRECT_RECEIVE, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObSort, PHY_SORT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObRootTransmit, PHY_ROOT_TRANSMIT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObDirectReceive, PHY_DIRECT_RECEIVE, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObRootTransmit, PHY_ROOT_TRANSMIT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
cur_op = tmp_op;
|
||||
|
||||
physical_plan->set_main_query(cur_op);
|
||||
|
||||
char buf[3000];
|
||||
int64_t pos = 0;
|
||||
ObJobParserTest::print_phy_op_tree(buf, 3000 - 1, pos, cur_op);
|
||||
_OB_LOG(INFO, "physical operator tree:\n%s", buf);
|
||||
|
||||
ObExecContext exec_ctx;
|
||||
exec_ctx.init_phy_op(1);
|
||||
exec_ctx.create_physical_plan_ctx();
|
||||
ObDistributedJobControl jc;
|
||||
ObAddrsProviderFactory spf;
|
||||
ObTaskSpliterFactory tsf;
|
||||
ObJobParser parser;
|
||||
ASSERT_EQ(OB_INVALID_ARGUMENT, parser.parse_job(exec_ctx, NULL, OB_INVALID_ID, tsf, spf, jc));
|
||||
ASSERT_EQ(OB_SUCCESS, parser.parse_job(exec_ctx, physical_plan, OB_INVALID_ID, tsf, spf, jc));
|
||||
ASSERT_EQ(3, jc.get_job_count());
|
||||
ObJob *tmp_job = NULL;
|
||||
ObArray<ObJob *> jobs;
|
||||
jc.get_all_jobs(jobs);
|
||||
tmp_job = jobs.at(0);
|
||||
ASSERT_EQ(3, tmp_job->get_priority());
|
||||
tmp_job = jobs.at(1);
|
||||
ASSERT_EQ(2, tmp_job->get_priority());
|
||||
tmp_job = jobs.at(2);
|
||||
ASSERT_EQ(1, tmp_job->get_priority());
|
||||
|
||||
char buf1[3000];
|
||||
int64_t pos1 = 0;
|
||||
ObJobParserTest::print_job_tree(buf1, 3000 - 1, pos1, &jc);
|
||||
_OB_LOG(INFO, "job tree:\n%s", buf1);
|
||||
|
||||
jobs.reset();
|
||||
jc.get_ready_jobs(jobs);
|
||||
ASSERT_EQ(1, jobs.count());
|
||||
ASSERT_EQ(3, jobs.at(0)->get_priority());
|
||||
//ASSERT_EQ(3, jobs.at(1)->get_priority());
|
||||
//ASSERT_EQ(3, jobs.at(2)->get_priority());
|
||||
//ASSERT_EQ(3, jobs.at(3)->get_priority());
|
||||
}
|
||||
|
||||
TEST_F(ObJobParserTest, error_test_1)
|
||||
{
|
||||
ObPhysicalPlan *physical_plan = ObPhysicalPlan::alloc();
|
||||
ObPhyOperator *cur_op = NULL;
|
||||
ObPhyOperator *tmp_op = NULL;
|
||||
int err_code = OB_SUCCESS;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObTableScan, PHY_TABLE_SCAN, physical_plan, err_code);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObSort, PHY_SORT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObSort, PHY_SORT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObRootTransmit, PHY_ROOT_TRANSMIT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObDirectReceive, PHY_DIRECT_RECEIVE, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObSort, PHY_SORT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObRootTransmit, PHY_ROOT_TRANSMIT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObDirectReceive, PHY_DIRECT_RECEIVE, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObRootTransmit, PHY_ROOT_TRANSMIT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
cur_op = tmp_op;
|
||||
|
||||
physical_plan->set_main_query(cur_op);
|
||||
|
||||
char buf[3000];
|
||||
int64_t pos = 0;
|
||||
ObJobParserTest::print_phy_op_tree(buf, 3000 - 1, pos, cur_op);
|
||||
_OB_LOG(INFO, "physical operator tree:\n%s", buf);
|
||||
|
||||
ObExecContext exec_ctx;
|
||||
exec_ctx.init_phy_op(1);
|
||||
exec_ctx.create_physical_plan_ctx();
|
||||
ObDistributedJobControl jc;
|
||||
ObAddrsProviderFactory spf;
|
||||
ObTaskSpliterFactory tsf;
|
||||
ObJobParser parser;
|
||||
ASSERT_EQ(OB_INVALID_ARGUMENT, parser.parse_job(exec_ctx, NULL, OB_INVALID_ID, tsf, spf, jc));
|
||||
ASSERT_EQ(OB_SUCCESS, parser.parse_job(exec_ctx, physical_plan, OB_INVALID_ID, tsf, spf, jc));
|
||||
}
|
||||
|
||||
TEST_F(ObJobParserTest, error_test_2)
|
||||
{
|
||||
ObPhysicalPlan *physical_plan = ObPhysicalPlan::alloc();
|
||||
ObPhyOperator *cur_op = NULL;
|
||||
ObPhyOperator *tmp_op = NULL;
|
||||
int err_code = OB_SUCCESS;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObTableScan, PHY_TABLE_SCAN, physical_plan, err_code);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObSort, PHY_SORT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObSort, PHY_SORT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObRootTransmit, PHY_ROOT_TRANSMIT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObDirectReceive, PHY_DIRECT_RECEIVE, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObSort, PHY_SORT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObRootTransmit, PHY_ROOT_TRANSMIT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObDirectReceive, PHY_DIRECT_RECEIVE, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObRootTransmit, PHY_ROOT_TRANSMIT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
cur_op = tmp_op;
|
||||
|
||||
physical_plan->set_main_query(cur_op);
|
||||
|
||||
char buf[3000];
|
||||
int64_t pos = 0;
|
||||
ObJobParserTest::print_phy_op_tree(buf, 3000 - 1, pos, cur_op);
|
||||
_OB_LOG(INFO, "physical operator tree:\n%s", buf);
|
||||
|
||||
ObExecContext exec_ctx;
|
||||
exec_ctx.init_phy_op(1);
|
||||
exec_ctx.create_physical_plan_ctx();
|
||||
ObDistributedJobControl jc;
|
||||
ObAddrsProviderFactory spf;
|
||||
ObTaskSpliterFactory tsf;
|
||||
ObJobParser parser;
|
||||
ASSERT_EQ(OB_INVALID_ARGUMENT, parser.parse_job(exec_ctx, NULL, OB_INVALID_ID, tsf, spf, jc));
|
||||
ASSERT_EQ(OB_SUCCESS, parser.parse_job(exec_ctx, physical_plan, OB_INVALID_ID, tsf, spf, jc));
|
||||
}
|
||||
|
||||
TEST_F(ObJobParserTest, error_test_3)
|
||||
{
|
||||
ObPhysicalPlan *physical_plan = ObPhysicalPlan::alloc();
|
||||
ObPhyOperator *cur_op = NULL;
|
||||
ObPhyOperator *tmp_op = NULL;
|
||||
int err_code = OB_SUCCESS;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObTableScan, PHY_TABLE_SCAN, physical_plan, err_code);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObSort, PHY_SORT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObSort, PHY_SORT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObRootTransmit, PHY_ROOT_TRANSMIT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObDirectReceive, PHY_DIRECT_RECEIVE, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObSort, PHY_SORT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObRootTransmit, PHY_ROOT_TRANSMIT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObDirectReceive, PHY_DIRECT_RECEIVE, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
cur_op = tmp_op;
|
||||
|
||||
TEST_CREATE_PHY_OPERATOR(tmp_op, ObRootTransmit, PHY_ROOT_TRANSMIT, physical_plan, err_code);
|
||||
tmp_op->set_child(0, *cur_op);
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::IDENTITY_SPLIT);
|
||||
cur_op = tmp_op;
|
||||
|
||||
physical_plan->set_main_query(cur_op);
|
||||
|
||||
char buf[3000];
|
||||
int64_t pos = 0;
|
||||
ObJobParserTest::print_phy_op_tree(buf, 3000 - 1, pos, cur_op);
|
||||
_OB_LOG(INFO, "physical operator tree:\n%s", buf);
|
||||
|
||||
ObExecContext exec_ctx;
|
||||
exec_ctx.init_phy_op(1); // actually should be 9, the operator count
|
||||
exec_ctx.create_physical_plan_ctx();
|
||||
ObDistributedJobControl jc;
|
||||
ObAddrsProviderFactory spf;
|
||||
ObTaskSpliterFactory tsf;
|
||||
ObJobParser parser;
|
||||
ASSERT_EQ(OB_INVALID_ARGUMENT, parser.parse_job(exec_ctx, NULL, OB_INVALID_ID, tsf, spf, jc));
|
||||
ASSERT_EQ(OB_SUCCESS, parser.parse_job(exec_ctx, physical_plan, OB_INVALID_ID, tsf, spf, jc));
|
||||
}
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
oceanbase::common::ObLogger::get_logger().set_log_level("DEBUG");
|
||||
init_sql_factories();
|
||||
::testing::InitGoogleTest(&argc,argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
||||
198
unittest/sql/executor/test_job_parser_perf.cpp
Normal file
198
unittest/sql/executor/test_job_parser_perf.cpp
Normal file
@ -0,0 +1,198 @@
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
#include "lib/allocator/page_arena.h"
|
||||
#include "sql/ob_sql_init.h"
|
||||
#include "create_op_util.h"
|
||||
//#include "sql/engine/basic/ob_project.h"
|
||||
//#include "sql/engine/join/ob_merge_join.h"
|
||||
//#include "sql/engine/dml/ob_when_filter.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/executor/ob_root_transmit.h"
|
||||
#include "sql/executor/ob_direct_receive.h"
|
||||
#include "sql/engine/table/ob_table_scan.h"
|
||||
#include "sql/executor/ob_job_parser.h"
|
||||
#include "sql/executor/ob_task_spliter_factory.h"
|
||||
#include "sql/plan_cache/ob_cache_object_factory.h"
|
||||
using namespace oceanbase;
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
|
||||
class ObJobParserPerfTest : public ::testing::Test
|
||||
{
|
||||
public:
|
||||
ObJobParserPerfTest();
|
||||
virtual ~ObJobParserPerfTest();
|
||||
virtual void SetUp();
|
||||
virtual void TearDown();
|
||||
static void print_phy_op_tree(char *buf, const int64_t buf_len, int64_t& pos, ObPhyOperator *op);
|
||||
static void print_sub_phy_op_tree(char *buf, const int64_t buf_len, int64_t& pos, ObPhyOperator *op);
|
||||
static void print_job_tree(char *buf, const int64_t buf_len, int64_t& pos, ObJobControl *jc);
|
||||
private:
|
||||
// disallow copy
|
||||
ObJobParserPerfTest(const ObJobParserPerfTest &other);
|
||||
ObJobParserPerfTest& operator=(const ObJobParserPerfTest &ohter);
|
||||
private:
|
||||
static void print_op_tree(char *buf, const int64_t buf_len, int64_t& pos, ObPhyOperator *op, int32_t level);
|
||||
static void print_sub_op_tree(char *buf, const int64_t buf_len, int64_t& pos, ObPhyOperator *op, int32_t level);
|
||||
};
|
||||
|
||||
|
||||
ObJobParserPerfTest::ObJobParserPerfTest()
|
||||
{
|
||||
}
|
||||
|
||||
ObJobParserPerfTest::~ObJobParserPerfTest()
|
||||
{
|
||||
}
|
||||
|
||||
void ObJobParserPerfTest::SetUp()
|
||||
{
|
||||
}
|
||||
|
||||
void ObJobParserPerfTest::TearDown()
|
||||
{
|
||||
}
|
||||
|
||||
void ObJobParserPerfTest::print_phy_op_tree(char *buf, const int64_t buf_len, int64_t& pos, ObPhyOperator *op)
|
||||
{
|
||||
ObJobParserPerfTest::print_op_tree(buf, buf_len, pos, op, 0);
|
||||
buf[pos] = '\0';
|
||||
}
|
||||
|
||||
void ObJobParserPerfTest::print_op_tree(char *buf, const int64_t buf_len, int64_t& pos, ObPhyOperator *op, int32_t level)
|
||||
{
|
||||
const char* op_name = NULL;
|
||||
if (NULL != op)
|
||||
{
|
||||
op_name = ob_phy_operator_type_str(op->get_type());
|
||||
}
|
||||
for (int32_t i = 0; i < level; i++)
|
||||
{
|
||||
::oceanbase::common::databuff_printf(buf, buf_len, pos, " ");
|
||||
}
|
||||
::oceanbase::common::databuff_printf(buf, buf_len, pos, "|- %s\n", op_name);
|
||||
if (NULL != op)
|
||||
{
|
||||
for (int32_t i = 0; i < op->get_child_num(); i++)
|
||||
{
|
||||
ObJobParserPerfTest::print_op_tree(buf, buf_len, pos, op->get_child(i), level + 1);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void ObJobParserPerfTest::print_job_tree(char *buf, const int64_t buf_len, int64_t& pos, ObJobControl *jc)
|
||||
{
|
||||
::oceanbase::common::databuff_printf(buf, buf_len, pos, "total jobs: %ld\n", jc->get_job_count());
|
||||
ObArray<ObJob *> jobs;
|
||||
jc->get_all_jobs(jobs);
|
||||
for (int64_t i = 0; i < jobs.count(); ++i)
|
||||
{
|
||||
ObJob *job = jobs.at(i);
|
||||
::oceanbase::common::databuff_printf(buf, buf_len, pos, "job_id=%ld\n", i);
|
||||
ObJobParserPerfTest::print_sub_phy_op_tree(buf, buf_len, pos, job->get_root_op());
|
||||
}
|
||||
buf[pos] = '\0';
|
||||
}
|
||||
|
||||
void ObJobParserPerfTest::print_sub_phy_op_tree(char *buf, const int64_t buf_len, int64_t& pos, ObPhyOperator *op)
|
||||
{
|
||||
ObJobParserPerfTest::print_sub_op_tree(buf, buf_len, pos, op, 0);
|
||||
buf[pos] = '\0';
|
||||
}
|
||||
|
||||
void ObJobParserPerfTest::print_sub_op_tree(char *buf, const int64_t buf_len, int64_t& pos, ObPhyOperator *op, int32_t level)
|
||||
{
|
||||
const char* op_name = NULL;
|
||||
if (NULL != op)
|
||||
{
|
||||
op_name = ob_phy_operator_type_str(op->get_type());
|
||||
}
|
||||
for (int32_t i = 0; i < level; i++)
|
||||
{
|
||||
::oceanbase::common::databuff_printf(buf, buf_len, pos, " ");
|
||||
}
|
||||
::oceanbase::common::databuff_printf(buf, buf_len, pos, "|- %s\n", op_name);
|
||||
if (NULL != op)
|
||||
{
|
||||
for (int32_t i = 0; i < op->get_child_num(); i++)
|
||||
{
|
||||
ObTransmit* trans_op = dynamic_cast<ObTransmit*>(op->get_child(i));
|
||||
if (NULL == trans_op)
|
||||
{
|
||||
ObJobParserPerfTest::print_sub_op_tree(buf, buf_len, pos, op->get_child(i), level + 1);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
TEST_F(ObJobParserPerfTest, basic_test)
|
||||
{
|
||||
ObPhysicalPlan tmp_plan;
|
||||
UNUSED(tmp_plan);
|
||||
ObPhysicalPlan *physical_plan =NULL;
|
||||
ObCacheObjectFactory::alloc(physical_plan);
|
||||
ObPhyOperator *cur_op = NULL;
|
||||
ObPhyOperator *tmp_op = NULL;
|
||||
|
||||
ASSERT_FALSE(NULL == physical_plan);
|
||||
ASSERT_EQ(OB_SUCCESS, physical_plan->alloc_operator_by_type(PHY_TABLE_SCAN, tmp_op));
|
||||
static_cast<ObTransmit*>(tmp_op)->get_job_conf().set_task_split_type(ObTaskSpliter::LOCAL_IDENTITY_SPLIT);
|
||||
cur_op = tmp_op;
|
||||
|
||||
physical_plan->set_main_query(cur_op);
|
||||
|
||||
char buf[3000];
|
||||
int64_t pos = 0;
|
||||
ObJobParserPerfTest::print_phy_op_tree(buf, 3000 - 1, pos, cur_op);
|
||||
_OB_LOG(INFO, "physical operator tree:\n%s", buf);
|
||||
|
||||
int counter = 0;
|
||||
int64_t sum = 0;
|
||||
while(counter < 10000) {
|
||||
counter++;
|
||||
ObExecContext exec_ctx;
|
||||
exec_ctx.init_phy_op(1);
|
||||
exec_ctx.create_physical_plan_ctx();
|
||||
ObLocalJobControl jc;
|
||||
ObAddrsProviderFactory spf;
|
||||
ObTaskSpliterFactory tsf;
|
||||
ObJobParser parser;
|
||||
ObExecutionID execution_id;
|
||||
int64_t start = ::oceanbase::common::ObTimeUtility::current_time();
|
||||
ASSERT_EQ(OB_SUCCESS, parser.parse_job(exec_ctx, physical_plan, execution_id, tsf, spf, jc));
|
||||
int64_t end = ::oceanbase::common::ObTimeUtility::current_time();
|
||||
sum += (end - start);
|
||||
|
||||
ASSERT_EQ(1, jc.get_job_count());
|
||||
// ObJob *tmp_job = NULL;
|
||||
ObArray<ObJob *> jobs;
|
||||
// jc.get_all_jobs(jobs);
|
||||
// tmp_job = jobs.at(0);
|
||||
// ASSERT_EQ(1, tmp_job->get_priority());
|
||||
|
||||
jobs.reset();
|
||||
jc.get_ready_jobs(jobs);
|
||||
ASSERT_EQ(1, jobs.count());
|
||||
// ASSERT_EQ(1, jobs.at(0)->get_priority());
|
||||
}
|
||||
_OB_LOG(INFO, "time cost %ld", sum * 1000 / counter);
|
||||
}
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
oceanbase::common::ObLogger::get_logger().set_log_level("INFO");
|
||||
init_sql_factories();
|
||||
::testing::InitGoogleTest(&argc,argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
||||
200
unittest/sql/executor/test_slice_id.cpp
Normal file
200
unittest/sql/executor/test_slice_id.cpp
Normal file
@ -0,0 +1,200 @@
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
#include "sql/executor/ob_slice_id.h"
|
||||
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
|
||||
class ObSliceIDTest : public ::testing::Test
|
||||
{
|
||||
public:
|
||||
ObSliceIDTest();
|
||||
virtual ~ObSliceIDTest();
|
||||
virtual void SetUp();
|
||||
virtual void TearDown();
|
||||
private:
|
||||
// disallow copy
|
||||
ObSliceIDTest(const ObSliceIDTest &other);
|
||||
ObSliceIDTest& operator=(const ObSliceIDTest &other);
|
||||
private:
|
||||
// data members
|
||||
};
|
||||
ObSliceIDTest::ObSliceIDTest()
|
||||
{
|
||||
}
|
||||
|
||||
ObSliceIDTest::~ObSliceIDTest()
|
||||
{
|
||||
}
|
||||
|
||||
void ObSliceIDTest::SetUp()
|
||||
{
|
||||
}
|
||||
|
||||
void ObSliceIDTest::TearDown()
|
||||
{
|
||||
}
|
||||
|
||||
TEST_F(ObSliceIDTest, all_test)
|
||||
{
|
||||
const static uint64_t valid_id_a = 1033;
|
||||
const static uint64_t valid_id_b = 42233;
|
||||
const static uint64_t valid_id_c = 423;
|
||||
const static uint64_t valid_id_d = 1;
|
||||
ObSliceID slice_id;
|
||||
ObSliceID slice_id2;
|
||||
ObJobID job_id;
|
||||
ObTaskID task_id;
|
||||
ObAddr server(ObAddr::IPV4, "127.0.0.1", 8080);
|
||||
|
||||
// initial state
|
||||
ASSERT_EQ(OB_INVALID_ID, slice_id.get_execution_id());
|
||||
ASSERT_EQ(OB_INVALID_ID, slice_id.get_job_id());
|
||||
ASSERT_EQ(OB_INVALID_ID, slice_id.get_slice_id());
|
||||
ASSERT_TRUE(job_id.equal(slice_id.get_ob_job_id()));
|
||||
ASSERT_TRUE(task_id.equal(slice_id.get_ob_task_id()));
|
||||
|
||||
// state change
|
||||
slice_id.set_server(server);
|
||||
slice_id.set_execution_id(valid_id_a);
|
||||
slice_id.set_job_id(valid_id_b);
|
||||
slice_id.set_task_id(valid_id_c);
|
||||
slice_id.set_slice_id(valid_id_d);
|
||||
ASSERT_TRUE(slice_id.is_valid());
|
||||
|
||||
// state change
|
||||
ASSERT_FALSE(slice_id2.is_valid());
|
||||
slice_id2.set_server(server);
|
||||
ASSERT_FALSE(slice_id2.is_valid());
|
||||
slice_id2.set_execution_id(valid_id_a);
|
||||
ASSERT_FALSE(slice_id2.is_valid());
|
||||
slice_id2.set_job_id(valid_id_b);
|
||||
ASSERT_FALSE(slice_id2.is_valid());
|
||||
slice_id2.set_task_id(valid_id_c);
|
||||
ASSERT_FALSE(slice_id2.is_valid());
|
||||
slice_id2.set_slice_id(valid_id_d);
|
||||
ASSERT_TRUE(slice_id2.is_valid());
|
||||
|
||||
|
||||
ASSERT_EQ(valid_id_a, slice_id.get_execution_id());
|
||||
ASSERT_EQ(valid_id_b, slice_id.get_job_id());
|
||||
ASSERT_EQ(valid_id_c, slice_id.get_task_id());
|
||||
ASSERT_EQ(valid_id_d, slice_id.get_slice_id());
|
||||
ASSERT_EQ(server, slice_id.get_server());
|
||||
|
||||
ASSERT_TRUE(slice_id.equal(slice_id2));
|
||||
ASSERT_TRUE(slice_id.hash() == slice_id2.hash());
|
||||
|
||||
slice_id.reset();
|
||||
ASSERT_FALSE(slice_id.equal(slice_id2));
|
||||
slice_id2.reset();
|
||||
ASSERT_TRUE(slice_id.equal(slice_id2));
|
||||
|
||||
|
||||
}
|
||||
|
||||
TEST_F(ObSliceIDTest, serialize_test)
|
||||
{
|
||||
ObSliceID slice_id;
|
||||
ObSliceID slice_id2;
|
||||
|
||||
slice_id.set_execution_id(OB_INVALID_ID);
|
||||
slice_id.set_job_id(OB_INVALID_ID);
|
||||
slice_id.set_slice_id(OB_INVALID_ID);
|
||||
|
||||
// invalid val deserialize
|
||||
char buf[1024];
|
||||
int64_t buf_len = 1024;
|
||||
int64_t pos = 0;
|
||||
int64_t pos2 = 0;
|
||||
int ret = slice_id.serialize(buf, buf_len, pos);
|
||||
EXPECT_TRUE(OB_SUCC(ret));
|
||||
slice_id2.deserialize(buf, pos, pos2);
|
||||
EXPECT_TRUE(true == slice_id.equal(slice_id2));
|
||||
|
||||
// valid val deserialize
|
||||
uint64_t valid_id_a = 1012;
|
||||
uint64_t valid_id_b = 13012;
|
||||
uint64_t valid_id_c = 130120;
|
||||
uint64_t valid_id_d = 1;
|
||||
ObAddr server(ObAddr::IPV4, "127.0.0.1", 8080);
|
||||
|
||||
slice_id.set_execution_id(valid_id_a);
|
||||
slice_id.set_job_id(valid_id_b);
|
||||
slice_id.set_task_id(valid_id_c);
|
||||
slice_id.set_slice_id(valid_id_d);
|
||||
slice_id.set_server(server);
|
||||
ret = slice_id.serialize(buf, buf_len, pos);
|
||||
EXPECT_TRUE(OB_SUCC(ret));
|
||||
slice_id2.deserialize(buf, pos, pos2);
|
||||
EXPECT_TRUE(true == slice_id.is_valid());
|
||||
EXPECT_TRUE(true == slice_id2.is_valid());
|
||||
EXPECT_TRUE(true == slice_id.equal(slice_id2));
|
||||
|
||||
EXPECT_TRUE(slice_id.get_serialize_size() == slice_id2.get_serialize_size());
|
||||
SQL_EXE_LOG(INFO, "info:", K(slice_id));
|
||||
SQL_EXE_LOG(INFO, "info:", "size", slice_id.get_serialize_size());
|
||||
}
|
||||
|
||||
TEST_F(ObSliceIDTest, hash_test)
|
||||
{
|
||||
ObJobID job_id;
|
||||
ObSliceID slice_id;
|
||||
ObSliceID slice_id2;
|
||||
uint64_t valid_id_a = 1012;
|
||||
uint64_t valid_id_b = 13012;
|
||||
uint64_t valid_id_c = 130120;
|
||||
|
||||
slice_id.set_execution_id(valid_id_a);
|
||||
slice_id.set_job_id(valid_id_b);
|
||||
slice_id.set_slice_id(valid_id_c);
|
||||
|
||||
slice_id2.set_execution_id(valid_id_a);
|
||||
slice_id2.set_job_id(valid_id_b);
|
||||
slice_id2.set_slice_id(valid_id_c);
|
||||
|
||||
ASSERT_TRUE(slice_id.hash() == slice_id2.hash());
|
||||
slice_id.reset();
|
||||
slice_id2.reset();
|
||||
ASSERT_TRUE(slice_id.hash() == slice_id2.hash());
|
||||
|
||||
}
|
||||
|
||||
|
||||
TEST_F(ObSliceIDTest, struct_test)
|
||||
{
|
||||
ObJobID job_id;
|
||||
ObSliceID slice_id;
|
||||
uint64_t valid_id_a = 1012;
|
||||
uint64_t valid_id_b = 13012;
|
||||
uint64_t valid_id_c = 130120;
|
||||
|
||||
slice_id.set_execution_id(valid_id_a);
|
||||
slice_id.set_job_id(valid_id_b);
|
||||
slice_id.set_slice_id(valid_id_c);
|
||||
|
||||
job_id = slice_id.get_ob_job_id();
|
||||
EXPECT_TRUE(job_id.get_execution_id() == valid_id_a);
|
||||
EXPECT_TRUE(job_id.get_job_id() == valid_id_b);
|
||||
EXPECT_TRUE(job_id.get_job_id() == slice_id.get_job_id());
|
||||
EXPECT_TRUE(job_id.get_execution_id() == slice_id.get_execution_id());
|
||||
|
||||
}
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
oceanbase::common::ObLogger::get_logger().set_log_level("INFO");
|
||||
::testing::InitGoogleTest(&argc,argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
||||
159
unittest/sql/executor/test_task_event.cpp
Normal file
159
unittest/sql/executor/test_task_event.cpp
Normal file
@ -0,0 +1,159 @@
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include "sql/executor/ob_task_event.h"
|
||||
#include <gtest/gtest.h>
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
|
||||
class ObTaskEventTest : public ::testing::Test
|
||||
{
|
||||
public:
|
||||
ObTaskEventTest();
|
||||
virtual ~ObTaskEventTest();
|
||||
virtual void SetUp();
|
||||
virtual void TearDown();
|
||||
private:
|
||||
// disallow copy
|
||||
ObTaskEventTest(const ObTaskEventTest &other);
|
||||
ObTaskEventTest& operator=(const ObTaskEventTest &other);
|
||||
private:
|
||||
// data members
|
||||
};
|
||||
ObTaskEventTest::ObTaskEventTest()
|
||||
{
|
||||
}
|
||||
|
||||
ObTaskEventTest::~ObTaskEventTest()
|
||||
{
|
||||
}
|
||||
|
||||
void ObTaskEventTest::SetUp()
|
||||
{
|
||||
}
|
||||
|
||||
void ObTaskEventTest::TearDown()
|
||||
{
|
||||
}
|
||||
|
||||
TEST_F(ObTaskEventTest, all_test)
|
||||
{
|
||||
const static uint64_t valid_id_a = 1033;
|
||||
const static uint64_t valid_id_b = 42233;
|
||||
const static uint64_t valid_id_c = 423;
|
||||
const static int64_t err_code_a = static_cast<int64_t>(OB_NOT_SUPPORTED);
|
||||
ObTaskEvent task_event;
|
||||
ObTaskEvent task_event2;
|
||||
ObTaskLocation task_loc;
|
||||
ObTaskLocation task_loc2;
|
||||
ObJobID job_id;
|
||||
ObTaskID task_id;
|
||||
ObAddr server(ObAddr::IPV4, "127.0.0.1", 8080);
|
||||
ObAddr ctrl_server(ObAddr::IPV4, "127.0.0.1", 8080);
|
||||
|
||||
// initial state
|
||||
ASSERT_FALSE(task_event.is_valid());
|
||||
ASSERT_EQ(OB_INVALID_ARGUMENT, task_event.init(task_loc, err_code_a));
|
||||
ASSERT_FALSE(task_event.is_valid());
|
||||
ASSERT_EQ(OB_INVALID_ID, task_loc.get_execution_id());
|
||||
ASSERT_EQ(OB_INVALID_ID, task_loc.get_job_id());
|
||||
ASSERT_EQ(OB_INVALID_ID, task_loc.get_task_id());
|
||||
ASSERT_TRUE(job_id.equal(task_loc.get_ob_job_id()));
|
||||
ASSERT_TRUE(task_id.equal(task_loc.get_ob_task_id()));
|
||||
ASSERT_TRUE(task_event.get_task_location().equal(task_loc));
|
||||
ASSERT_TRUE(task_event.get_task_location().equal(task_loc2));
|
||||
ASSERT_FALSE(task_event.get_err_code() == err_code_a);
|
||||
|
||||
// state change
|
||||
task_loc.set_server(server);
|
||||
task_loc.set_ctrl_server(ctrl_server);
|
||||
task_loc.set_execution_id(valid_id_a);
|
||||
task_loc.set_job_id(valid_id_b);
|
||||
task_loc.set_task_id(valid_id_c);
|
||||
ASSERT_TRUE(task_loc.is_valid());
|
||||
ASSERT_EQ(OB_SUCCESS, task_event.init(task_loc, err_code_a));
|
||||
ASSERT_TRUE(task_event.is_valid());
|
||||
ASSERT_TRUE(task_event.get_task_location().equal(task_loc));
|
||||
ASSERT_FALSE(task_event.get_task_location().equal(task_loc2));
|
||||
ASSERT_TRUE(task_event.get_err_code() == err_code_a);
|
||||
|
||||
// assign
|
||||
ObArenaAllocator allocator(ObModIds::OB_SQL_EXECUTOR_TASK_EVENT);
|
||||
ASSERT_FALSE(task_event.equal(task_event2));
|
||||
ASSERT_FALSE(task_event2.is_valid());
|
||||
ASSERT_EQ(OB_SUCCESS, task_event2.assign(allocator, task_event));
|
||||
ASSERT_TRUE(task_event.equal(task_event2));
|
||||
ASSERT_TRUE(task_event2.is_valid());
|
||||
|
||||
task_event.reset();
|
||||
ASSERT_FALSE(task_event.equal(task_event2));
|
||||
ASSERT_FALSE(task_event.get_task_location().equal(task_loc));
|
||||
ASSERT_TRUE(task_event.get_task_location().equal(task_loc2));
|
||||
ASSERT_FALSE(task_event.get_err_code() == err_code_a);
|
||||
ASSERT_FALSE(task_event.is_valid());
|
||||
ASSERT_TRUE(task_event2.is_valid());
|
||||
task_event2.reset();
|
||||
ASSERT_TRUE(task_event.equal(task_event2));
|
||||
ASSERT_FALSE(task_event.is_valid());
|
||||
ASSERT_FALSE(task_event2.is_valid());
|
||||
}
|
||||
|
||||
TEST_F(ObTaskEventTest, serialize_test)
|
||||
{
|
||||
const static uint64_t valid_id_a = 1033;
|
||||
const static uint64_t valid_id_b = 42233;
|
||||
const static uint64_t valid_id_c = 423;
|
||||
const static uint64_t err_code_a = static_cast<int64_t>(OB_NOT_SUPPORTED);
|
||||
ObAddr server(ObAddr::IPV4, "127.0.0.1", 8080);
|
||||
ObAddr ctrl_server(ObAddr::IPV4, "127.0.0.1", 8080);
|
||||
|
||||
ObTaskEvent task_event;
|
||||
ObTaskEvent task_event2;
|
||||
ObTaskLocation task_loc;
|
||||
|
||||
// invalid val deserialize
|
||||
char buf[1024];
|
||||
int64_t buf_len = 1024;
|
||||
int64_t pos = 0;
|
||||
int64_t pos2 = 0;
|
||||
int ret = task_event.serialize(buf, buf_len, pos);
|
||||
EXPECT_TRUE(OB_SUCC(ret));
|
||||
task_event2.deserialize(buf, pos, pos2);
|
||||
EXPECT_FALSE(true == task_event.is_valid());
|
||||
EXPECT_FALSE(true == task_event2.is_valid());
|
||||
ASSERT_TRUE(task_event.equal(task_event2));
|
||||
|
||||
// valid val deserialize
|
||||
task_loc.set_server(server);
|
||||
task_loc.set_ctrl_server(ctrl_server);
|
||||
task_loc.set_execution_id(valid_id_a);
|
||||
task_loc.set_job_id(valid_id_b);
|
||||
task_loc.set_task_id(valid_id_c);
|
||||
ASSERT_EQ(OB_SUCCESS, task_event.init(task_loc, err_code_a));
|
||||
ret = task_event.serialize(buf, buf_len, pos);
|
||||
EXPECT_TRUE(OB_SUCC(ret));
|
||||
task_event2.deserialize(buf, pos, pos2);
|
||||
EXPECT_TRUE(true == task_event.is_valid());
|
||||
EXPECT_TRUE(true == task_event2.is_valid());
|
||||
ASSERT_TRUE(task_event.equal(task_event2));
|
||||
|
||||
EXPECT_TRUE(task_event.get_serialize_size() == task_event2.get_serialize_size());
|
||||
SQL_EXE_LOG(INFO, "info:", K(task_event));
|
||||
SQL_EXE_LOG(INFO, "info:", "size", task_event.get_serialize_size());
|
||||
}
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
oceanbase::common::ObLogger::get_logger().set_log_level("INFO");
|
||||
::testing::InitGoogleTest(&argc,argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
||||
234
unittest/sql/executor/test_task_id.cpp
Normal file
234
unittest/sql/executor/test_task_id.cpp
Normal file
@ -0,0 +1,234 @@
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
#include "sql/executor/ob_task_id.h"
|
||||
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
|
||||
class ObTaskIDTest : public ::testing::Test
|
||||
{
|
||||
public:
|
||||
ObTaskIDTest();
|
||||
virtual ~ObTaskIDTest();
|
||||
virtual void SetUp();
|
||||
virtual void TearDown();
|
||||
private:
|
||||
// disallow copy
|
||||
ObTaskIDTest(const ObTaskIDTest &other);
|
||||
ObTaskIDTest& operator=(const ObTaskIDTest &other);
|
||||
private:
|
||||
// data members
|
||||
};
|
||||
ObTaskIDTest::ObTaskIDTest()
|
||||
{
|
||||
}
|
||||
|
||||
ObTaskIDTest::~ObTaskIDTest()
|
||||
{
|
||||
}
|
||||
|
||||
void ObTaskIDTest::SetUp()
|
||||
{
|
||||
}
|
||||
|
||||
void ObTaskIDTest::TearDown()
|
||||
{
|
||||
}
|
||||
|
||||
TEST_F(ObTaskIDTest, basic_test)
|
||||
{
|
||||
const static uint64_t valid_id_a = 1033;
|
||||
const static uint64_t valid_id_b = 42233;
|
||||
const static uint64_t valid_id_c = 423;
|
||||
ObTaskID task_id;
|
||||
|
||||
// initial state
|
||||
ASSERT_EQ(OB_INVALID_ID, task_id.get_execution_id());
|
||||
ASSERT_EQ(OB_INVALID_ID, task_id.get_job_id());
|
||||
ASSERT_EQ(OB_INVALID_ID, task_id.get_task_id());
|
||||
|
||||
// state change
|
||||
task_id.set_execution_id(valid_id_a);
|
||||
task_id.set_job_id(valid_id_b);
|
||||
task_id.set_task_id(valid_id_c);
|
||||
ASSERT_EQ(valid_id_a, task_id.get_execution_id());
|
||||
ASSERT_EQ(valid_id_b, task_id.get_job_id());
|
||||
ASSERT_EQ(valid_id_c, task_id.get_task_id());
|
||||
}
|
||||
|
||||
TEST_F(ObTaskIDTest, equal_test)
|
||||
{
|
||||
const static uint64_t valid_id_a = 1033;
|
||||
const static uint64_t valid_id_b = 42233;
|
||||
const static uint64_t valid_id_c = 423;
|
||||
ObTaskID task_id;
|
||||
ObTaskID task_id2;
|
||||
|
||||
// consist initial state
|
||||
ASSERT_EQ(true, task_id.equal(task_id2));
|
||||
ASSERT_EQ(true, task_id2.equal(task_id));
|
||||
|
||||
// state change
|
||||
task_id.set_execution_id(valid_id_a);
|
||||
task_id.set_job_id(valid_id_b);
|
||||
task_id.set_task_id(valid_id_c);
|
||||
task_id2.set_execution_id(valid_id_a);
|
||||
task_id2.set_job_id(valid_id_b);
|
||||
task_id2.set_task_id(valid_id_c);
|
||||
ASSERT_EQ(true, task_id.equal(task_id2));
|
||||
ASSERT_EQ(true, task_id2.equal(task_id));
|
||||
|
||||
// state change
|
||||
task_id.set_execution_id(valid_id_a);
|
||||
task_id.set_job_id(valid_id_b);
|
||||
task_id.set_task_id(valid_id_c);
|
||||
task_id2.set_execution_id(valid_id_a);
|
||||
task_id2.set_job_id(valid_id_a);
|
||||
task_id2.set_task_id(valid_id_a);
|
||||
EXPECT_TRUE(false == task_id.equal(task_id2));
|
||||
EXPECT_TRUE(false == task_id2.equal(task_id));
|
||||
|
||||
task_id.reset();
|
||||
task_id2.reset();
|
||||
EXPECT_TRUE(false == task_id.is_valid());
|
||||
EXPECT_TRUE(false == task_id2.is_valid());
|
||||
EXPECT_TRUE(true == task_id.equal(task_id2));
|
||||
}
|
||||
|
||||
|
||||
TEST_F(ObTaskIDTest, valid_test)
|
||||
{
|
||||
const static uint64_t valid_id = 1;
|
||||
ObTaskID task_id;
|
||||
task_id.set_execution_id(OB_INVALID_ID);
|
||||
EXPECT_TRUE(false == task_id.is_valid());
|
||||
task_id.set_task_id(OB_INVALID_ID);
|
||||
EXPECT_TRUE(false == task_id.is_valid());
|
||||
|
||||
|
||||
task_id.set_execution_id(OB_INVALID_ID);
|
||||
task_id.set_task_id(valid_id);
|
||||
EXPECT_TRUE(false == task_id.is_valid());
|
||||
|
||||
|
||||
task_id.set_execution_id(valid_id);
|
||||
task_id.set_task_id(OB_INVALID_ID);
|
||||
EXPECT_TRUE(false == task_id.is_valid());
|
||||
|
||||
|
||||
task_id.set_execution_id(valid_id);
|
||||
task_id.set_job_id(valid_id);
|
||||
task_id.set_task_id(valid_id);
|
||||
EXPECT_TRUE(false == task_id.is_valid());
|
||||
|
||||
task_id.set_server(ObAddr(ObAddr::IPV4, "127.0.0.1", 8888));
|
||||
task_id.set_execution_id(valid_id);
|
||||
task_id.set_job_id(valid_id);
|
||||
task_id.set_task_id(valid_id);
|
||||
EXPECT_TRUE(true == task_id.is_valid());
|
||||
|
||||
}
|
||||
|
||||
TEST_F(ObTaskIDTest, serialize_test)
|
||||
{
|
||||
ObTaskID task_id;
|
||||
ObTaskID task_id2;
|
||||
|
||||
task_id.set_server(ObAddr(ObAddr::IPV4, "127.0.0.1", 8888));
|
||||
task_id.set_execution_id(OB_INVALID_ID);
|
||||
task_id.set_job_id(OB_INVALID_ID);
|
||||
task_id.set_task_id(OB_INVALID_ID);
|
||||
|
||||
// invalid val deserialize
|
||||
char buf[1024];
|
||||
int64_t buf_len = 1024;
|
||||
int64_t pos = 0;
|
||||
int64_t pos2 = 0;
|
||||
int ret = task_id.serialize(buf, buf_len, pos);
|
||||
EXPECT_TRUE(OB_SUCC(ret));
|
||||
task_id2.deserialize(buf, pos, pos2);
|
||||
EXPECT_TRUE(false == task_id.is_valid());
|
||||
EXPECT_TRUE(false == task_id2.is_valid());
|
||||
EXPECT_TRUE(true == task_id.equal(task_id2));
|
||||
|
||||
// valid val deserialize
|
||||
uint64_t valid_id_a = 1012;
|
||||
uint64_t valid_id_b = 13012;
|
||||
uint64_t valid_id_c = 130120;
|
||||
task_id.set_execution_id(valid_id_a);
|
||||
task_id.set_job_id(valid_id_b);
|
||||
task_id.set_task_id(valid_id_c);
|
||||
ret = task_id.serialize(buf, buf_len, pos);
|
||||
EXPECT_TRUE(OB_SUCC(ret));
|
||||
task_id2.deserialize(buf, pos, pos2);
|
||||
EXPECT_TRUE(true == task_id.is_valid());
|
||||
EXPECT_TRUE(true == task_id2.is_valid());
|
||||
EXPECT_TRUE(true == task_id.equal(task_id2));
|
||||
|
||||
EXPECT_TRUE(task_id.get_serialize_size() == task_id2.get_serialize_size());
|
||||
SQL_EXE_LOG(INFO, "info", K(task_id), "size", task_id.get_serialize_size());
|
||||
|
||||
}
|
||||
|
||||
TEST_F(ObTaskIDTest, hash_test)
|
||||
{
|
||||
ObJobID job_id;
|
||||
ObTaskID task_id;
|
||||
ObTaskID task_id2;
|
||||
uint64_t valid_id_a = 1012;
|
||||
uint64_t valid_id_b = 13012;
|
||||
uint64_t valid_id_c = 130120;
|
||||
|
||||
task_id.set_execution_id(valid_id_a);
|
||||
task_id.set_job_id(valid_id_b);
|
||||
task_id.set_task_id(valid_id_c);
|
||||
|
||||
task_id2.set_execution_id(valid_id_a);
|
||||
task_id2.set_job_id(valid_id_b);
|
||||
task_id2.set_task_id(valid_id_c);
|
||||
|
||||
ASSERT_TRUE(task_id.hash() == task_id2.hash());
|
||||
task_id.reset();
|
||||
task_id2.reset();
|
||||
ASSERT_TRUE(task_id.hash() == task_id2.hash());
|
||||
|
||||
}
|
||||
|
||||
|
||||
TEST_F(ObTaskIDTest, struct_test)
|
||||
{
|
||||
ObJobID job_id;
|
||||
ObTaskID task_id;
|
||||
uint64_t valid_id_a = 1012;
|
||||
uint64_t valid_id_b = 13012;
|
||||
uint64_t valid_id_c = 130120;
|
||||
|
||||
task_id.set_execution_id(valid_id_a);
|
||||
task_id.set_job_id(valid_id_b);
|
||||
task_id.set_task_id(valid_id_c);
|
||||
|
||||
job_id = task_id.get_ob_job_id();
|
||||
EXPECT_TRUE(job_id.get_execution_id() == valid_id_a);
|
||||
EXPECT_TRUE(job_id.get_job_id() == valid_id_b);
|
||||
EXPECT_TRUE(job_id.get_job_id() == task_id.get_job_id());
|
||||
EXPECT_TRUE(job_id.get_execution_id() == task_id.get_execution_id());
|
||||
|
||||
}
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
oceanbase::common::ObLogger::get_logger().set_log_level("INFO");
|
||||
::testing::InitGoogleTest(&argc,argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
||||
133
unittest/sql/executor/test_task_info.cpp
Normal file
133
unittest/sql/executor/test_task_info.cpp
Normal file
@ -0,0 +1,133 @@
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
#include "sql/executor/ob_task_info.h"
|
||||
#include "sql/executor/ob_fifo_receive.h"
|
||||
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
|
||||
class ObTaskInfoTest : public ::testing::Test
|
||||
{
|
||||
public:
|
||||
ObTaskInfoTest();
|
||||
virtual ~ObTaskInfoTest();
|
||||
virtual void SetUp();
|
||||
virtual void TearDown();
|
||||
void MakeSliceID(ObSliceID &slice_id) {
|
||||
int64_t valid_id_a = 1012;
|
||||
int64_t valid_id_b = 13012;
|
||||
int64_t valid_id_c = 130120;
|
||||
int64_t valid_id_d = 1;
|
||||
ObAddr server(ObAddr::IPV4, "127.0.0.1", 8080);
|
||||
slice_id.set_execution_id(valid_id_a);
|
||||
slice_id.set_job_id(valid_id_b);
|
||||
slice_id.set_task_id(valid_id_c);
|
||||
slice_id.set_slice_id(valid_id_d);
|
||||
slice_id.set_server(server);
|
||||
}
|
||||
|
||||
void MakeTaskID(ObTaskID &task_id, int64_t a, int64_t b=1001, int64_t c=1002, int64_t d=1) {
|
||||
task_id.set_execution_id(a);
|
||||
task_id.set_job_id(b);
|
||||
task_id.set_task_id(c);
|
||||
task_id.set_task_id(d);
|
||||
}
|
||||
|
||||
|
||||
void MakeServer(ObAddr &svr) {
|
||||
ObAddr server(ObAddr::IPV4, "127.0.0.1", 8080);
|
||||
svr = server;
|
||||
}
|
||||
|
||||
void MakeTaskID(ObTaskID &task_id) {
|
||||
int64_t valid_id_a = 1012;
|
||||
int64_t valid_id_b = 13012;
|
||||
int64_t valid_id_c = 130120;
|
||||
int64_t valid_id_d = 1;
|
||||
task_id.set_execution_id(valid_id_a);
|
||||
task_id.set_job_id(valid_id_b);
|
||||
task_id.set_task_id(valid_id_c);
|
||||
task_id.set_task_id(valid_id_d);
|
||||
}
|
||||
|
||||
private:
|
||||
// disallow copy
|
||||
ObTaskInfoTest(const ObTaskInfoTest &other);
|
||||
ObTaskInfoTest& operator=(const ObTaskInfoTest &other);
|
||||
private:
|
||||
// data members
|
||||
};
|
||||
ObTaskInfoTest::ObTaskInfoTest()
|
||||
{
|
||||
}
|
||||
|
||||
ObTaskInfoTest::~ObTaskInfoTest()
|
||||
{
|
||||
}
|
||||
|
||||
void ObTaskInfoTest::SetUp()
|
||||
{
|
||||
}
|
||||
|
||||
void ObTaskInfoTest::TearDown()
|
||||
{
|
||||
}
|
||||
|
||||
TEST_F(ObTaskInfoTest, all_test)
|
||||
{
|
||||
ObArenaAllocator allocator(ObModIds::TEST);
|
||||
ObTaskInfo ti(allocator);
|
||||
|
||||
// state test
|
||||
ASSERT_TRUE(OB_TASK_STATE_NOT_INIT == ti.get_state());
|
||||
ti.set_state(OB_TASK_STATE_INITED);
|
||||
ASSERT_TRUE(OB_TASK_STATE_INITED == ti.get_state());
|
||||
|
||||
ObArenaAllocator alloc;
|
||||
// root op test
|
||||
ObPhyOperator *op = new ObFifoReceive(alloc);
|
||||
ASSERT_TRUE(NULL == ti.get_root_op());
|
||||
ti.set_root_op(op);
|
||||
ASSERT_TRUE(op == ti.get_root_op());
|
||||
|
||||
// range test
|
||||
// TODO
|
||||
//
|
||||
|
||||
// location test
|
||||
ObTaskLocation loc;
|
||||
ObAddr server;
|
||||
ObTaskID task_id;
|
||||
ObTaskLocation loc_tmp;
|
||||
ObAddr server_tmp;
|
||||
ObTaskID task_id_tmp;
|
||||
this->MakeTaskID(task_id);
|
||||
this->MakeServer(server);
|
||||
loc.set_ob_task_id(task_id);
|
||||
loc.set_server(server);
|
||||
ti.set_task_location(loc);
|
||||
loc_tmp = ti.get_task_location();
|
||||
task_id_tmp = loc_tmp.get_ob_task_id();
|
||||
server_tmp = loc_tmp.get_server();
|
||||
ASSERT_TRUE(task_id_tmp.equal(task_id));
|
||||
ASSERT_TRUE(server.hash() == server_tmp.hash());
|
||||
|
||||
}
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
oceanbase::common::ObLogger::get_logger().set_log_level("INFO");
|
||||
::testing::InitGoogleTest(&argc,argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
||||
215
unittest/sql/executor/test_task_location.cpp
Normal file
215
unittest/sql/executor/test_task_location.cpp
Normal file
@ -0,0 +1,215 @@
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
|
||||
#include "sql/executor/ob_task_location.h"
|
||||
#include <gtest/gtest.h>
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
|
||||
class ObTaskLocationTest : public ::testing::Test
|
||||
{
|
||||
public:
|
||||
ObTaskLocationTest();
|
||||
virtual ~ObTaskLocationTest();
|
||||
virtual void SetUp();
|
||||
virtual void TearDown();
|
||||
private:
|
||||
// disallow copy
|
||||
ObTaskLocationTest(const ObTaskLocationTest &other);
|
||||
ObTaskLocationTest& operator=(const ObTaskLocationTest &other);
|
||||
private:
|
||||
// data members
|
||||
};
|
||||
ObTaskLocationTest::ObTaskLocationTest()
|
||||
{
|
||||
}
|
||||
|
||||
ObTaskLocationTest::~ObTaskLocationTest()
|
||||
{
|
||||
}
|
||||
|
||||
void ObTaskLocationTest::SetUp()
|
||||
{
|
||||
}
|
||||
|
||||
void ObTaskLocationTest::TearDown()
|
||||
{
|
||||
}
|
||||
|
||||
TEST_F(ObTaskLocationTest, all_test)
|
||||
{
|
||||
const static uint64_t valid_id_a = 1033;
|
||||
const static uint64_t valid_id_b = 42233;
|
||||
const static uint64_t valid_id_c = 423;
|
||||
ObTaskLocation task_loc;
|
||||
ObTaskLocation task_loc2;
|
||||
ObTaskLocation task_loc3;
|
||||
ObJobID job_id;
|
||||
ObTaskID task_id;
|
||||
ObAddr server(ObAddr::IPV4, "127.0.0.1", 8080);
|
||||
ObAddr ctrl_server(ObAddr::IPV4, "127.0.0.1", 8888);
|
||||
|
||||
// initial state
|
||||
ASSERT_EQ(OB_INVALID_ID, task_loc.get_execution_id());
|
||||
ASSERT_EQ(OB_INVALID_ID, task_loc.get_job_id());
|
||||
ASSERT_EQ(OB_INVALID_ID, task_loc.get_task_id());
|
||||
ASSERT_TRUE(job_id.equal(task_loc.get_ob_job_id()));
|
||||
ASSERT_TRUE(task_id.equal(task_loc.get_ob_task_id()));
|
||||
|
||||
// state change
|
||||
task_loc.set_server(server);
|
||||
task_loc.set_ctrl_server(ctrl_server);
|
||||
task_loc.set_execution_id(valid_id_a);
|
||||
task_loc.set_job_id(valid_id_b);
|
||||
task_loc.set_task_id(valid_id_c);
|
||||
ASSERT_TRUE(task_loc.is_valid());
|
||||
|
||||
// state change
|
||||
ASSERT_FALSE(task_loc2.is_valid());
|
||||
task_loc2.set_server(server);
|
||||
ASSERT_FALSE(task_loc2.is_valid());
|
||||
task_loc2.set_ctrl_server(ctrl_server);
|
||||
ASSERT_FALSE(task_loc2.is_valid());
|
||||
task_loc2.set_execution_id(valid_id_a);
|
||||
ASSERT_FALSE(task_loc2.is_valid());
|
||||
task_loc2.set_job_id(valid_id_b);
|
||||
ASSERT_FALSE(task_loc2.is_valid());
|
||||
task_loc2.set_task_id(valid_id_c);
|
||||
ASSERT_TRUE(task_loc2.is_valid());
|
||||
|
||||
// assign
|
||||
ASSERT_FALSE(task_loc3.equal(task_loc));
|
||||
task_loc3 = task_loc;
|
||||
ASSERT_TRUE(task_loc3.equal(task_loc));
|
||||
|
||||
ASSERT_EQ(valid_id_a, task_loc.get_execution_id());
|
||||
ASSERT_EQ(valid_id_b, task_loc.get_job_id());
|
||||
ASSERT_EQ(valid_id_c, task_loc.get_task_id());
|
||||
ASSERT_EQ(server, task_loc.get_server());
|
||||
|
||||
ASSERT_TRUE(task_loc.equal(task_loc2));
|
||||
ASSERT_TRUE(task_loc.hash() == task_loc2.hash());
|
||||
|
||||
task_loc.reset();
|
||||
ASSERT_FALSE(task_loc.equal(task_loc2));
|
||||
task_loc2.reset();
|
||||
ASSERT_TRUE(task_loc.equal(task_loc2));
|
||||
}
|
||||
|
||||
TEST_F(ObTaskLocationTest, serialize_test)
|
||||
{
|
||||
ObTaskLocation task_loc;
|
||||
ObTaskLocation task_loc2;
|
||||
|
||||
task_loc.set_execution_id(OB_INVALID_ID);
|
||||
task_loc.set_job_id(OB_INVALID_ID);
|
||||
task_loc.set_task_id(OB_INVALID_ID);
|
||||
|
||||
// invalid val deserialize
|
||||
char buf[1024];
|
||||
int64_t buf_len = 1024;
|
||||
int64_t pos = 0;
|
||||
int64_t pos2 = 0;
|
||||
int ret = task_loc.serialize(buf, buf_len, pos);
|
||||
EXPECT_TRUE(OB_SUCC(ret));
|
||||
task_loc2.deserialize(buf, pos, pos2);
|
||||
EXPECT_TRUE(true == task_loc.equal(task_loc2));
|
||||
|
||||
// valid val deserialize
|
||||
uint64_t valid_id_a = 1012;
|
||||
uint64_t valid_id_b = 13012;
|
||||
uint64_t valid_id_c = 130120;
|
||||
ObAddr server(ObAddr::IPV4, "127.0.0.1", 8080);
|
||||
ObAddr ctrl_server(ObAddr::IPV4, "127.0.0.1", 8888);
|
||||
|
||||
task_loc.set_server(server);
|
||||
task_loc.set_ctrl_server(ctrl_server);
|
||||
task_loc.set_execution_id(valid_id_a);
|
||||
task_loc.set_job_id(valid_id_b);
|
||||
task_loc.set_task_id(valid_id_c);
|
||||
task_loc.set_server(server);
|
||||
ret = task_loc.serialize(buf, buf_len, pos);
|
||||
EXPECT_TRUE(OB_SUCC(ret));
|
||||
task_loc2.deserialize(buf, pos, pos2);
|
||||
EXPECT_TRUE(true == task_loc.is_valid());
|
||||
EXPECT_TRUE(true == task_loc2.is_valid());
|
||||
EXPECT_TRUE(true == task_loc.equal(task_loc2));
|
||||
|
||||
EXPECT_TRUE(task_loc.get_serialize_size() == task_loc2.get_serialize_size());
|
||||
SQL_EXE_LOG(INFO, "info:", K(task_loc));
|
||||
SQL_EXE_LOG(INFO, "info:", "size", task_loc.get_serialize_size());
|
||||
}
|
||||
|
||||
TEST_F(ObTaskLocationTest, hash_test)
|
||||
{
|
||||
ObJobID job_id;
|
||||
ObTaskLocation task_loc;
|
||||
ObTaskLocation task_loc2;
|
||||
uint64_t valid_id_a = 1012;
|
||||
uint64_t valid_id_b = 13012;
|
||||
uint64_t valid_id_c = 130120;
|
||||
ObAddr server(ObAddr::IPV4, "127.0.0.1", 8080);
|
||||
|
||||
task_loc.set_server(server);
|
||||
task_loc.set_execution_id(valid_id_a);
|
||||
task_loc.set_job_id(valid_id_b);
|
||||
task_loc.set_task_id(valid_id_c);
|
||||
|
||||
task_loc2.set_server(server);
|
||||
task_loc2.set_execution_id(valid_id_a);
|
||||
task_loc2.set_job_id(valid_id_b);
|
||||
task_loc2.set_task_id(valid_id_c);
|
||||
|
||||
ASSERT_TRUE(task_loc.hash() == task_loc2.hash());
|
||||
task_loc.reset();
|
||||
task_loc2.reset();
|
||||
ASSERT_TRUE(task_loc.hash() == task_loc2.hash());
|
||||
}
|
||||
|
||||
|
||||
TEST_F(ObTaskLocationTest, struct_test)
|
||||
{
|
||||
ObJobID job_id;
|
||||
ObTaskID task_id;
|
||||
ObTaskLocation task_loc;
|
||||
uint64_t valid_id_a = 1012;
|
||||
uint64_t valid_id_b = 13012;
|
||||
uint64_t valid_id_c = 130120;
|
||||
ObAddr server(ObAddr::IPV4, "127.0.0.1", 8080);
|
||||
|
||||
task_loc.set_server(server);
|
||||
task_loc.set_execution_id(valid_id_a);
|
||||
task_loc.set_job_id(valid_id_b);
|
||||
task_loc.set_task_id(valid_id_c);
|
||||
|
||||
job_id = task_loc.get_ob_job_id();
|
||||
EXPECT_TRUE(job_id.get_execution_id() == valid_id_a);
|
||||
EXPECT_TRUE(job_id.get_job_id() == valid_id_b);
|
||||
EXPECT_TRUE(job_id.get_execution_id() == task_loc.get_execution_id());
|
||||
EXPECT_TRUE(job_id.get_job_id() == task_loc.get_job_id());
|
||||
|
||||
|
||||
task_id = task_loc.get_ob_task_id();
|
||||
EXPECT_TRUE(task_id.get_execution_id() == valid_id_a);
|
||||
EXPECT_TRUE(task_id.get_job_id() == valid_id_b);
|
||||
EXPECT_TRUE(task_id.get_task_id() == valid_id_c);
|
||||
EXPECT_TRUE(task_id.get_execution_id() == task_loc.get_execution_id());
|
||||
EXPECT_TRUE(task_id.get_job_id() == task_loc.get_job_id());
|
||||
EXPECT_TRUE(task_id.get_task_id() == task_loc.get_task_id());
|
||||
}
|
||||
|
||||
int main(int argc, char **argv)
|
||||
{
|
||||
oceanbase::common::ObLogger::get_logger().set_log_level("INFO");
|
||||
::testing::InitGoogleTest(&argc,argv);
|
||||
return RUN_ALL_TESTS();
|
||||
}
|
||||
Reference in New Issue
Block a user