init push
This commit is contained in:
256
src/sql/engine/sequence/ob_sequence.cpp
Normal file
256
src/sql/engine/sequence/ob_sequence.cpp
Normal file
@ -0,0 +1,256 @@
|
||||
/**
|
||||
* 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_ENG
|
||||
#include "sql/engine/sequence/ob_sequence.h"
|
||||
#include "share/sequence/ob_sequence_cache.h"
|
||||
#include "lib/utility/utility.h"
|
||||
#include "share/object/ob_obj_cast.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/expr/ob_sql_expression.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
using namespace share;
|
||||
using namespace share::schema;
|
||||
namespace sql {
|
||||
class ObSequence::ObSequenceCtx : public ObPhyOperatorCtx {
|
||||
public:
|
||||
explicit ObSequenceCtx(ObExecContext& ctx) : ObPhyOperatorCtx(ctx), sequence_cache_(nullptr)
|
||||
{
|
||||
sequence_cache_ = &share::ObSequenceCache::get_instance();
|
||||
if (OB_ISNULL(sequence_cache_)) {
|
||||
LOG_ERROR("fail alloc memory for ObSequenceCache instance");
|
||||
}
|
||||
}
|
||||
|
||||
void reset()
|
||||
{
|
||||
sequence_cache_ = NULL;
|
||||
seq_schemas_.reset();
|
||||
}
|
||||
|
||||
virtual void destroy()
|
||||
{
|
||||
sequence_cache_ = NULL;
|
||||
seq_schemas_.reset();
|
||||
ObPhyOperatorCtx::destroy_base();
|
||||
}
|
||||
|
||||
private:
|
||||
share::ObSequenceCache* sequence_cache_;
|
||||
common::ObSEArray<ObSequenceSchema, 1> seq_schemas_;
|
||||
friend class ObSequence;
|
||||
};
|
||||
|
||||
ObSequence::ObSequence(ObIAllocator& alloc) : ObMultiChildrenPhyOperator(alloc), nextval_seq_ids_()
|
||||
{}
|
||||
|
||||
ObSequence::~ObSequence()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
|
||||
void ObSequence::reset()
|
||||
{
|
||||
ObMultiChildrenPhyOperator::reset();
|
||||
}
|
||||
|
||||
void ObSequence::reuse()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
|
||||
bool ObSequence::is_valid() const
|
||||
{
|
||||
bool bret = false;
|
||||
if (get_child_num() == 1) {
|
||||
bret = get_child(ObPhyOperator::FIRST_CHILD) != NULL && get_column_count() > 0 &&
|
||||
get_child(ObPhyOperator::FIRST_CHILD)->get_column_count() > 0;
|
||||
} else if (get_child_num() == 0) {
|
||||
bret = get_column_count() > 0;
|
||||
} else {
|
||||
// invalid
|
||||
}
|
||||
return bret;
|
||||
}
|
||||
|
||||
int ObSequence::add_uniq_nextval_sequence_id(uint64_t seq_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
FOREACH_X(sid, nextval_seq_ids_, OB_SUCC(ret))
|
||||
{
|
||||
if (seq_id == *sid) {
|
||||
ret = OB_ENTRY_EXIST;
|
||||
LOG_WARN("should not add duplicated seq id to ObSequence operator", K(seq_id), K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(nextval_seq_ids_.push_back(seq_id))) {
|
||||
LOG_WARN("fail add seq id to nextval seq id set", K(seq_id), K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSequence::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSequenceCtx* sequence_ctx = NULL;
|
||||
if (OB_UNLIKELY(!is_valid())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("sequence operator is invalid", K_(nextval_seq_ids));
|
||||
} else if (OB_FAIL(init_op_ctx(ctx))) {
|
||||
LOG_WARN("initialize operator context failed", K(ret));
|
||||
} else if (OB_ISNULL(sequence_ctx = GET_PHY_OPERATOR_CTX(ObSequenceCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K_(id));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSequence::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
ObSequenceCtx* sequence_ctx = NULL;
|
||||
if (OB_NOT_NULL(sequence_ctx = GET_PHY_OPERATOR_CTX(ObSequenceCtx, ctx, get_id()))) {
|
||||
sequence_ctx->reset();
|
||||
}
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
int ObSequence::init_op_ctx(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSequenceCtx* op_ctx = NULL;
|
||||
ObTaskExecutorCtx* task_ctx = NULL;
|
||||
share::schema::ObMultiVersionSchemaService* schema_service = NULL;
|
||||
ObSQLSessionInfo* my_session = NULL;
|
||||
share::schema::ObSchemaGetterGuard schema_guard;
|
||||
if (OB_ISNULL(my_session = GET_MY_SESSION(ctx))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get my session", K(ret));
|
||||
} else if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObSequenceCtx, ctx, get_id(), get_type(), op_ctx))) {
|
||||
LOG_WARN("failed to create SequenceCtx", K(ret));
|
||||
} else if (OB_ISNULL(op_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("op_ctx is null");
|
||||
} else if (OB_FAIL(init_cur_row(*op_ctx, need_copy_row_for_compute()))) {
|
||||
LOG_WARN("init current row failed", K(ret));
|
||||
} else if (OB_ISNULL(task_ctx = GET_TASK_EXECUTOR_CTX(ctx))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("task executor ctx is null", K(ret));
|
||||
} else if (OB_ISNULL(schema_service = task_ctx->schema_service_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("schema service is null", K(ret));
|
||||
} else if (OB_FAIL(schema_service->get_tenant_schema_guard(my_session->get_effective_tenant_id(), schema_guard))) {
|
||||
LOG_WARN("get schema guard failed", K(ret));
|
||||
} else {
|
||||
uint64_t tenant_id = my_session->get_effective_tenant_id();
|
||||
const ObIArray<uint64_t>& ids = nextval_seq_ids_;
|
||||
ARRAY_FOREACH_X(ids, idx, cnt, OB_SUCC(ret))
|
||||
{
|
||||
const uint64_t seq_id = ids.at(idx);
|
||||
const ObSequenceSchema* seq_schema = nullptr;
|
||||
if (OB_FAIL(schema_guard.get_sequence_schema(tenant_id, seq_id, seq_schema))) {
|
||||
LOG_WARN("fail get sequence schema", K(seq_id), K(ret));
|
||||
} else if (OB_ISNULL(seq_schema)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("null unexpected", K(ret));
|
||||
} else if (OB_FAIL(op_ctx->seq_schemas_.push_back(*seq_schema))) {
|
||||
LOG_WARN("cache seq_schema fail", K(tenant_id), K(seq_id), K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSequence::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSequenceCtx* sequence_ctx = NULL;
|
||||
ObSQLSessionInfo* my_session = NULL;
|
||||
const ObIArray<uint64_t>& ids = nextval_seq_ids_;
|
||||
if (OB_ISNULL(sequence_ctx = GET_PHY_OPERATOR_CTX(ObSequenceCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator ctx failed");
|
||||
} else if (OB_ISNULL(sequence_ctx->sequence_cache_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("seq cache not init", K(ret));
|
||||
} else if (OB_ISNULL(my_session = GET_MY_SESSION(ctx))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get my session", K(ret));
|
||||
} else if (OB_FAIL(try_get_next_row(ctx, row))) {
|
||||
LOG_WARN_IGNORE_ITER_END(ret, "fail get next row", K(ret));
|
||||
} else if (ids.count() != sequence_ctx->seq_schemas_.count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("id count does not match schema count",
|
||||
"id_cnt",
|
||||
ids.count(),
|
||||
"schema_cnt",
|
||||
sequence_ctx->seq_schemas_.count(),
|
||||
K(ret));
|
||||
} else {
|
||||
uint64_t tenant_id = my_session->get_effective_tenant_id();
|
||||
ObArenaAllocator allocator; // nextval temporary calculation memory
|
||||
ARRAY_FOREACH_X(ids, idx, cnt, OB_SUCC(ret))
|
||||
{
|
||||
const uint64_t seq_id = ids.at(idx);
|
||||
ObSequenceValue seq_value;
|
||||
if (OB_FAIL(sequence_ctx->sequence_cache_->nextval(sequence_ctx->seq_schemas_.at(idx), allocator, seq_value))) {
|
||||
LOG_WARN("fail get nextval for seq", K(tenant_id), K(seq_id), K(ret));
|
||||
} else if (OB_FAIL(my_session->set_sequence_value(tenant_id, seq_id, seq_value))) {
|
||||
LOG_WARN(
|
||||
"save seq_value to session as currval for later read fail", K(tenant_id), K(seq_id), K(seq_value), K(ret));
|
||||
} else {
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSequence::try_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPhyOperator* child = nullptr;
|
||||
ObSequenceCtx* sequence_ctx = NULL;
|
||||
if (get_child_num() > 1) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("should not have more than 1 child", K(ret));
|
||||
} else if (OB_ISNULL(sequence_ctx = GET_PHY_OPERATOR_CTX(ObSequenceCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator ctx failed");
|
||||
} else if (get_child_num() == 0) {
|
||||
// insert stmt, no child, give an empty row
|
||||
row = &(sequence_ctx->get_cur_row());
|
||||
} else if (OB_ISNULL(child = get_child(ObPhyOperator::FIRST_CHILD))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("child should not be null", K(ret));
|
||||
} else if (OB_FAIL(child->get_next_row(ctx, row))) {
|
||||
LOG_WARN_IGNORE_ITER_END(ret, "fail get next row", K(ret));
|
||||
} else if (OB_FAIL(copy_cur_row_by_projector(*sequence_ctx, row))) {
|
||||
LOG_WARN("copy current row failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSequence::add_filter(ObSqlExpression* expr)
|
||||
{
|
||||
UNUSED(expr);
|
||||
LOG_ERROR("sequence operator should have no filter expr");
|
||||
return OB_NOT_SUPPORTED;
|
||||
}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObSequence, ObMultiChildrenPhyOperator), nextval_seq_ids_);
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
77
src/sql/engine/sequence/ob_sequence.h
Normal file
77
src/sql/engine/sequence/ob_sequence.h
Normal file
@ -0,0 +1,77 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef _OB_SEQUENCE_H
|
||||
#define _OB_SEQUENCE_H 1
|
||||
#include "sql/engine/ob_multi_children_phy_operator.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObSqlExpression;
|
||||
class ObSequence : public ObMultiChildrenPhyOperator {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
private:
|
||||
class ObSequenceCtx;
|
||||
|
||||
public:
|
||||
explicit ObSequence(common::ObIAllocator& alloc);
|
||||
virtual ~ObSequence();
|
||||
|
||||
virtual void reset();
|
||||
virtual void reuse();
|
||||
int add_uniq_nextval_sequence_id(uint64_t seq_id);
|
||||
|
||||
private:
|
||||
bool is_valid() const;
|
||||
/**
|
||||
* @brief init operator context, will create a physical operator context (and a current row space)
|
||||
* @param ctx[in], execute context
|
||||
* @return if success, return OB_SUCCESS, otherwise, return errno
|
||||
*/
|
||||
int init_op_ctx(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief: called by get_next_row(), get a row from the child operator or row_store
|
||||
* @param: ctx[in], execute context
|
||||
* @param: row[out], ObSqlRow an obj array and row_size
|
||||
*/
|
||||
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
/**
|
||||
* @brief open operator, not including children operators.
|
||||
* called by open.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_open(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief close operator, not including children operators.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_close(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief overload add_filter to prevent any filter expression add to sequence
|
||||
* @param expr[in] any expr
|
||||
* @return always return OB_NOT_SUPPORTED
|
||||
*/
|
||||
int add_filter(ObSqlExpression* expr);
|
||||
|
||||
int try_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
TO_STRING_KV(K_(nextval_seq_ids));
|
||||
// disallow copy
|
||||
DISALLOW_COPY_AND_ASSIGN(ObSequence);
|
||||
|
||||
private:
|
||||
common::ObSEArray<uint64_t, 4> nextval_seq_ids_;
|
||||
};
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* _OB_SEQUENCE_H */
|
||||
188
src/sql/engine/sequence/ob_sequence_op.cpp
Normal file
188
src/sql/engine/sequence/ob_sequence_op.cpp
Normal file
@ -0,0 +1,188 @@
|
||||
/**
|
||||
* 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_ENG
|
||||
#include "sql/engine/sequence/ob_sequence_op.h"
|
||||
#include "lib/utility/utility.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
using namespace share;
|
||||
using namespace share::schema;
|
||||
namespace sql {
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObSequenceSpec, ObOpSpec), nextval_seq_ids_);
|
||||
|
||||
ObSequenceSpec::ObSequenceSpec(ObIAllocator& alloc, const ObPhyOperatorType type)
|
||||
: ObOpSpec(alloc, type), nextval_seq_ids_(alloc)
|
||||
{}
|
||||
|
||||
int ObSequenceSpec::add_uniq_nextval_sequence_id(uint64_t seq_id)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
for (uint64_t i = 0; i < nextval_seq_ids_.count() && OB_SUCC(ret); ++i) {
|
||||
if (seq_id == nextval_seq_ids_.at(i)) {
|
||||
ret = OB_ENTRY_EXIST;
|
||||
LOG_WARN("should not add duplicated seq id to ObSequence operator", K(seq_id), K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(nextval_seq_ids_.push_back(seq_id))) {
|
||||
LOG_WARN("fail add seq id to nextval seq id set", K(seq_id), K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
ObSequenceOp::ObSequenceOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObOperator(exec_ctx, spec, input), sequence_cache_(nullptr)
|
||||
{
|
||||
sequence_cache_ = &share::ObSequenceCache::get_instance();
|
||||
if (OB_ISNULL(sequence_cache_)) {
|
||||
LOG_ERROR("fail alloc memory for ObSequenceCache instance");
|
||||
}
|
||||
}
|
||||
|
||||
ObSequenceOp::~ObSequenceOp()
|
||||
{}
|
||||
|
||||
bool ObSequenceOp::is_valid()
|
||||
{
|
||||
bool bret = false;
|
||||
if (get_child_cnt() == 1) {
|
||||
bret = get_child() != NULL && spec_.output_.count() > 0 && get_child()->get_spec().output_.count() > 0;
|
||||
} else if (get_child_cnt() == 0) {
|
||||
bret = spec_.output_.count() > 0;
|
||||
} else {
|
||||
// invalid
|
||||
}
|
||||
return bret;
|
||||
}
|
||||
|
||||
int ObSequenceOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(!is_valid())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("sequence operator is invalid", K(MY_SPEC.nextval_seq_ids_));
|
||||
} else if (OB_FAIL(init_op())) {
|
||||
LOG_WARN("initialize operator context failed", K(ret));
|
||||
} else if (OB_ISNULL(sequence_cache_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("seq cache not init", K(ret));
|
||||
} else if (get_child_cnt() > 1) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("should not have more than 1 child", K(ret));
|
||||
} else if (0 < MY_SPEC.filters_.count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("sequence operator should have no filter expr", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSequenceOp::inner_close()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
reset();
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSequenceOp::init_op()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObTaskExecutorCtx* task_ctx = NULL;
|
||||
share::schema::ObMultiVersionSchemaService* schema_service = NULL;
|
||||
ObSQLSessionInfo* my_session = NULL;
|
||||
share::schema::ObSchemaGetterGuard schema_guard;
|
||||
if (OB_ISNULL(my_session = GET_MY_SESSION(ctx_))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get my session", K(ret));
|
||||
} else if (OB_ISNULL(task_ctx = GET_TASK_EXECUTOR_CTX(ctx_))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("task executor ctx is null", K(ret));
|
||||
} else if (OB_ISNULL(schema_service = task_ctx->schema_service_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("schema service is null", K(ret));
|
||||
} else if (OB_FAIL(schema_service->get_tenant_schema_guard(my_session->get_effective_tenant_id(), schema_guard))) {
|
||||
LOG_WARN("get schema guard failed", K(ret));
|
||||
} else {
|
||||
uint64_t tenant_id = my_session->get_effective_tenant_id();
|
||||
const ObIArray<uint64_t>& ids = MY_SPEC.nextval_seq_ids_;
|
||||
ARRAY_FOREACH_X(ids, idx, cnt, OB_SUCC(ret))
|
||||
{
|
||||
const uint64_t seq_id = ids.at(idx);
|
||||
const ObSequenceSchema* seq_schema = nullptr;
|
||||
if (OB_FAIL(schema_guard.get_sequence_schema(tenant_id, seq_id, seq_schema))) {
|
||||
LOG_WARN("fail get sequence schema", K(seq_id), K(ret));
|
||||
} else if (OB_ISNULL(seq_schema)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("null unexpected", K(ret));
|
||||
} else if (OB_FAIL(seq_schemas_.push_back(*seq_schema))) {
|
||||
LOG_WARN("cache seq_schema fail", K(tenant_id), K(seq_id), K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSequenceOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSQLSessionInfo* my_session = GET_MY_SESSION(ctx_);
|
||||
const ObIArray<uint64_t>& ids = MY_SPEC.nextval_seq_ids_;
|
||||
if (OB_FAIL(try_get_next_row())) {
|
||||
LOG_WARN_IGNORE_ITER_END(ret, "fail get next row", K(ret));
|
||||
} else if (ids.count() != seq_schemas_.count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("id count does not match schema count", "id_cnt", ids.count(), "schema_cnt", seq_schemas_.count(), K(ret));
|
||||
} else {
|
||||
uint64_t tenant_id = my_session->get_effective_tenant_id();
|
||||
ObArenaAllocator allocator;
|
||||
// need to update the nextval in the cache if and only if there is a nextval in the select item
|
||||
// otherwise, directly use the value in the session
|
||||
ARRAY_FOREACH_X(ids, idx, cnt, OB_SUCC(ret))
|
||||
{
|
||||
const uint64_t seq_id = ids.at(idx);
|
||||
// int64_t dummy_seq_value = 10240012435;
|
||||
ObSequenceValue seq_value;
|
||||
// Note: the order of schema and the order of id in ids are one-to-one correspondence
|
||||
// so you can directly use the subscript to address
|
||||
if (OB_FAIL(sequence_cache_->nextval(seq_schemas_.at(idx), allocator, seq_value))) {
|
||||
LOG_WARN("fail get nextval for seq", K(tenant_id), K(seq_id), K(ret));
|
||||
} else if (OB_FAIL(my_session->set_sequence_value(tenant_id, seq_id, seq_value))) {
|
||||
LOG_WARN(
|
||||
"save seq_value to session as currval for later read fail", K(tenant_id), K(seq_id), K(seq_value), K(ret));
|
||||
} else {
|
||||
// LOG_INFO("next seq value from sequence cache, saved to session", K(idx), K(seq_id), K(seq_value));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSequenceOp::try_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
clear_evaluated_flag();
|
||||
if (get_child_cnt() == 0) {
|
||||
// insert stmt, no child, give an empty row
|
||||
} else if (OB_FAIL(child_->get_next_row())) {
|
||||
LOG_WARN_IGNORE_ITER_END(ret, "fail get next row", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
76
src/sql/engine/sequence/ob_sequence_op.h
Normal file
76
src/sql/engine/sequence/ob_sequence_op.h
Normal file
@ -0,0 +1,76 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef _SRC_SQL_ENGINE_SEQENCE_OB_SEQUENCE_OP_H
|
||||
#define _SRC_SQL_ENGINE_SEQENCE_OB_SEQUENCE_OP_H 1
|
||||
#include "sql/engine/ob_operator.h"
|
||||
#include "share/sequence/ob_sequence_cache.h"
|
||||
#include "share/schema/ob_schema_struct.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
class ObSequenceSpec : public ObOpSpec {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObSequenceSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
|
||||
|
||||
INHERIT_TO_STRING_KV("op_spec", ObOpSpec, K_(nextval_seq_ids));
|
||||
|
||||
int add_uniq_nextval_sequence_id(uint64_t seq_id);
|
||||
common::ObFixedArray<uint64_t, common::ObIAllocator> nextval_seq_ids_;
|
||||
};
|
||||
|
||||
class ObSequenceOp : public ObOperator {
|
||||
public:
|
||||
ObSequenceOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
|
||||
~ObSequenceOp();
|
||||
|
||||
virtual int inner_get_next_row() override;
|
||||
virtual int inner_open() override;
|
||||
virtual int inner_close() override;
|
||||
|
||||
void reset()
|
||||
{
|
||||
sequence_cache_ = NULL;
|
||||
seq_schemas_.reset();
|
||||
}
|
||||
|
||||
virtual void destroy() override
|
||||
{
|
||||
sequence_cache_ = NULL;
|
||||
seq_schemas_.reset();
|
||||
ObOperator::destroy();
|
||||
}
|
||||
|
||||
private:
|
||||
bool is_valid();
|
||||
int init_op();
|
||||
/**
|
||||
* @brief overload add_filter to prevent any filter expression add to sequence
|
||||
* @param expr[in] any expr
|
||||
* @return always return OB_NOT_SUPPORTED
|
||||
*/
|
||||
int add_filter(ObSqlExpression* expr);
|
||||
|
||||
int try_get_next_row();
|
||||
|
||||
private:
|
||||
share::ObSequenceCache* sequence_cache_;
|
||||
common::ObSEArray<share::schema::ObSequenceSchema, 1> seq_schemas_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* _SRC_SQL_ENGINE_SEQENCE_OB_SEQUENCE_OP_H */
|
||||
Reference in New Issue
Block a user