patch 4.0

This commit is contained in:
wangzelin.wzl
2022-10-24 10:34:53 +08:00
parent 4ad6e00ec3
commit 93a1074b0c
10533 changed files with 2588271 additions and 2299373 deletions

View File

@ -11,27 +11,34 @@
*/
#define USING_LOG_PREFIX STORAGE
#include "ob_value_row_iterator.h"
#include "ob_partition_storage.h"
#include "ob_single_merge.h"
namespace oceanbase {
#include "share/schema/ob_table_dml_param.h"
#include "storage/ob_relative_table.h"
#include "storage/access/ob_single_merge.h"
#include "storage/ob_storage_table_guard.h"
#include "storage/tablet/ob_tablet.h"
#include "storage/ob_value_row_iterator.h"
#include "storage/memtable/ob_memtable_context.h"
namespace oceanbase
{
using namespace oceanbase::common;
namespace storage {
using namespace blocksstable;
namespace storage
{
ObValueRowIterator::ObValueRowIterator()
: ObNewRowIterator(),
is_inited_(false),
unique_(false),
allocator_(ObModIds::OB_VALUE_ROW_ITER),
rows_(),
cur_idx_(0),
data_table_rowkey_cnt_(0)
{}
cur_idx_(0)
{
}
ObValueRowIterator::~ObValueRowIterator()
{}
{
}
int ObValueRowIterator::init(bool unique, int64_t data_table_rowkey_cnt)
int ObValueRowIterator::init(bool unique)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(is_inited_)) {
@ -41,43 +48,38 @@ int ObValueRowIterator::init(bool unique, int64_t data_table_rowkey_cnt)
is_inited_ = true;
unique_ = unique;
cur_idx_ = 0;
data_table_rowkey_cnt_ = data_table_rowkey_cnt;
}
return ret;
}
int ObValueRowIterator::add_row(common::ObNewRow& row)
int ObValueRowIterator::add_row(common::ObNewRow &row)
{
int ret = OB_SUCCESS;
ObNewRow* cur_row = NULL;
ObNewRow *cur_row = NULL;
if (!row.is_valid()) {
ret = OB_INVALID_ARGUMENT;
STORAGE_LOG(WARN, "invalid row", K(ret), K(row));
} else if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
STORAGE_LOG(WARN, "ObValueRowIterator is not initialized", K(ret));
} else if (data_table_rowkey_cnt_ > row.count_) {
ret = OB_INVALID_ARGUMENT;
STORAGE_LOG(WARN, "invalid row", K(ret), K(row), K(data_table_rowkey_cnt_));
} else {
bool exist = false;
// check whether exists
if (unique_ && rows_.count() > 0) {
// we consider that in general, the probability that a row produces different conflicting rows
// on multiple unique index is small, so there is usually only one row in the value row iterator
// so using list traversal to deduplicate unique index is more efficiently
// and also saves the CPU overhead that constructs the hash map
ObStoreRowkey rowkey(row.cells_, data_table_rowkey_cnt_);
//we consider that in general, the probability that a row produces different conflicting rows
//on multiple unique index is small, so there is usually only one row in the value row iterator
//so using list traversal to deduplicate unique index is more efficiently
//and also saves the CPU overhead that constructs the hash map
ObStoreRowkey rowkey;
ObStoreRowkey tmp_rowkey;
if (OB_FAIL(rowkey.assign(row.cells_, row.count_))) {
STORAGE_LOG(WARN, "Failed to assign rowkey", K(ret), K(row));
}
for (int64_t i = 0; OB_SUCC(ret) && !exist && i < rows_.count(); ++i) {
if (data_table_rowkey_cnt_ > rows_.at(i).count_) {
ret = OB_INVALID_ARGUMENT;
STORAGE_LOG(WARN, "invalid row", K(ret), K(row), K(data_table_rowkey_cnt_));
} else {
ObStoreRowkey tmp_rowkey(rows_.at(i).cells_, data_table_rowkey_cnt_);
STORAGE_LOG(DEBUG, "print rowkey info", K(rowkey), K(tmp_rowkey), K(data_table_rowkey_cnt_));
if (OB_UNLIKELY(tmp_rowkey == rowkey)) {
exist = true;
}
if (OB_FAIL(tmp_rowkey.assign(rows_.at(i).cells_, rows_.at(i).count_))) {
STORAGE_LOG(WARN, "Failed to assign rowkey", K(ret), K(i), K(rows_.at(i)));
} else if (OB_UNLIKELY(tmp_rowkey == rowkey)) {
exist = true;
}
}
}
@ -87,7 +89,9 @@ int ObValueRowIterator::add_row(common::ObNewRow& row)
if (NULL == (cur_row = rows_.alloc_place_holder())) {
ret = OB_ALLOCATE_MEMORY_FAILED;
STORAGE_LOG(ERROR, "add row error", K(ret));
} else if (OB_SUCCESS != (ret = ob_write_row(allocator_, row, *cur_row))) {
} else if (OB_SUCCESS != (ret = ob_write_row(allocator_,
row,
*cur_row))) {
STORAGE_LOG(WARN, "copy row error", K(ret), K(row));
}
}
@ -96,7 +100,7 @@ int ObValueRowIterator::add_row(common::ObNewRow& row)
return ret;
}
int ObValueRowIterator::get_next_row(common::ObNewRow*& row)
int ObValueRowIterator::get_next_row(common::ObNewRow *&row)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!is_inited_)) {
@ -110,7 +114,7 @@ int ObValueRowIterator::get_next_row(common::ObNewRow*& row)
return ret;
}
int ObValueRowIterator::get_next_rows(ObNewRow*& rows, int64_t& row_count)
int ObValueRowIterator::get_next_rows(ObNewRow *&rows, int64_t &row_count)
{
int ret = OB_SUCCESS;
if (cur_idx_ < rows_.count()) {
@ -132,15 +136,16 @@ void ObValueRowIterator::reset()
cur_idx_ = 0;
}
ObSingleRowGetter::ObSingleRowGetter(ObIAllocator &allocator, ObPartitionStore &store)
: store_(store),
single_merge_(nullptr),
store_ctx_(nullptr),
output_projector_(allocator),
relative_table_(nullptr),
table_param_(nullptr),
allocator_(allocator)
{}
ObSingleRowGetter::ObSingleRowGetter(ObIAllocator &allocator, ObTablet &tablet)
: tablet_(&tablet),
single_merge_(nullptr),
store_ctx_(nullptr),
output_projector_(allocator),
relative_table_(nullptr),
table_param_(nullptr),
allocator_(allocator)
{
}
ObSingleRowGetter::~ObSingleRowGetter()
{
@ -156,68 +161,68 @@ ObSingleRowGetter::~ObSingleRowGetter()
}
}
int ObSingleRowGetter::init_dml_access_ctx(const ObStoreCtx &store_ctx, const ObDMLBaseParam &dml_param)
int ObSingleRowGetter::init_dml_access_ctx(
ObStoreCtx &store_ctx,
const ObDMLBaseParam &dml_param,
bool skip_read_lob)
{
int ret = OB_SUCCESS;
common::ObQueryFlag query_flag;
common::ObVersionRange trans_version_range;
// TODO (muwei) trans_version_range值后续由上层传入
trans_version_range.snapshot_version_ = store_ctx.mem_ctx_->get_read_snapshot();
query_flag.read_latest_ = ObQueryFlag::OBSF_MASK_READ_LATEST;
if (skip_read_lob) {
query_flag.skip_read_lob_ = ObQueryFlag::OBSF_MASK_SKIP_READ_LOB;
}
//TODO (yanfeng) trans_version_range值后续由上层传入
trans_version_range.snapshot_version_ = store_ctx.mvcc_acc_ctx_.get_snapshot_version();
trans_version_range.base_version_ = 0;
trans_version_range.multi_version_start_ = 0;
store_ctx_ = &store_ctx;
if (OB_FAIL(access_ctx_.init(dml_param.query_flag_, store_ctx, allocator_, trans_version_range))) {
if (OB_FAIL(access_ctx_.init(query_flag, store_ctx, allocator_, trans_version_range))) {
LOG_WARN("failed to init table access ctx", K(ret));
} else {
access_ctx_.expr_ctx_ = const_cast<ObExprCtx *>(&dml_param.expr_ctx_);
}
return ret;
}
int ObSingleRowGetter::init_dml_access_param(
ObRelativeTable &relative_table, const ObDMLBaseParam &dml_param, const ObIArray<uint64_t> &out_col_ids)
int ObSingleRowGetter::init_dml_access_param(ObRelativeTable &relative_table,
const ObDMLBaseParam &dml_param,
const ObIArray<uint64_t> &out_col_ids)
{
int ret = OB_SUCCESS;
relative_table_ = &relative_table;
get_table_param_.tables_handle_ = &(relative_table.tables_handle_);
if (!dml_param.virtual_columns_.empty() && !relative_table.is_index_table()) {
//The index table does not contain virtual columns, no need to set virtual_columns
access_param_.virtual_column_exprs_ = &(dml_param.virtual_columns_);
get_table_param_.tablet_iter_ = relative_table.tablet_iter_;
const share::schema::ObTableSchemaParam *schema_param = relative_table.get_schema_param();
output_projector_.set_capacity(out_col_ids.count());
for (int32_t i = 0; OB_SUCC(ret) && i < out_col_ids.count(); ++i) {
int idx = OB_INVALID_INDEX;
if (OB_FAIL(schema_param->get_col_map().get(out_col_ids.at(i), idx))) {
LOG_WARN("get column index from column map failed", K(ret), K(out_col_ids.at(i)));
} else if (OB_FAIL(output_projector_.push_back(idx))) {
LOG_WARN("store output projector failed", K(ret));
}
}
if (OB_UNLIKELY(!relative_table.use_schema_param())) {
const share::schema::ObTableSchema *schema = relative_table.get_schema();
if (OB_FAIL(create_table_param())) {
LOG_WARN("create table param failed", K(ret));
} else if (OB_FAIL(table_param_->convert(*schema, *schema, out_col_ids, false))) {
LOG_WARN("build table param from schema fail", K(ret), KPC(schema));
} else if (OB_FAIL(access_param_.init_dml_access_param(relative_table.get_table_id(),
relative_table.get_schema_version(),
relative_table.get_rowkey_column_num(),
*table_param_))) {
if (OB_SUCC(ret)) {
if (OB_FAIL(access_param_.init_dml_access_param(relative_table,
tablet_->get_full_read_info(),
*schema_param,
&output_projector_))) {
LOG_WARN("init dml access param failed", K(ret));
}
} else {
const share::schema::ObTableSchemaParam *schema_param = relative_table.get_schema_param();
output_projector_.set_capacity(out_col_ids.count());
for (int32_t i = 0; OB_SUCC(ret) && i < out_col_ids.count(); ++i) {
int idx = OB_INVALID_INDEX;
if (OB_FAIL(schema_param->get_col_map().get(out_col_ids.at(i), idx))) {
LOG_WARN("get column index from column map failed", K(ret), K(out_col_ids.at(i)));
} else if (OB_FAIL(output_projector_.push_back(idx))) {
LOG_WARN("store output projector failed", K(ret));
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(access_param_.init_dml_access_param(relative_table.get_table_id(),
relative_table.get_schema_version(),
relative_table.get_rowkey_column_num(),
*schema_param,
&output_projector_))) {
LOG_WARN("init dml access param failed", K(ret));
}
}
if (OB_SUCC(ret)) {
const ObTabletMeta &tablet_meta = tablet_->get_tablet_meta();
if (OB_UNLIKELY(!tablet_meta.is_valid())) {
LOG_WARN("tablet meta is invalid", K(ret), K(tablet_meta));
} else {
access_param_.iter_param_.tablet_id_ = tablet_meta.tablet_id_;
}
}
LOG_DEBUG("init dml access param", K(ret), K(out_col_ids), K(relative_table), K(dml_param), K_(access_param));
LOG_DEBUG("init dml access param", K(ret),
K(out_col_ids), K(relative_table),
K(dml_param), K_(access_param));
return ret;
}
@ -232,38 +237,36 @@ int ObSingleRowGetter::create_table_param()
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("allocate table param failed", K(ret), K(sizeof(share::schema::ObTableParam)));
} else {
table_param_ = new (buf) share::schema::ObTableParam(allocator_);
table_param_ = new(buf) share::schema::ObTableParam(allocator_);
}
return ret;
}
int ObSingleRowGetter::open(const ObStoreRowkey &rowkey, bool use_fuse_row_cache)
int ObSingleRowGetter::open(const ObDatumRowkey &rowkey, bool use_fuse_row_cache)
{
int ret = OB_SUCCESS;
void *buf = nullptr;
new (&ext_rowkey_) ObExtStoreRowkey(rowkey);
if (OB_ISNULL(buf = allocator_.alloc(sizeof(ObSingleMerge)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("Fail to allocate memory for multi get merge ", K(ret));
} else {
{
ObStorageWriterGuard guard(store_, *store_ctx_, false);
ObStorageTableGuard guard(tablet_, *store_ctx_, false);
if (OB_FAIL(guard.refresh_and_protect_table(*relative_table_))) {
STORAGE_LOG(WARN, "fail to protect table", K(ret));
}
}
single_merge_ = new (buf) ObSingleMerge();
single_merge_ = new(buf) ObSingleMerge();
}
if (OB_SUCC(ret)) {
if (OB_FAIL(single_merge_->init(access_param_, access_ctx_, get_table_param_))) {
STORAGE_LOG(WARN, "Fail to init ObSingleMerge, ", K(ret));
} else if (OB_FAIL(single_merge_->open(ext_rowkey_))) {
} else if (OB_FAIL(single_merge_->open(rowkey))) {
STORAGE_LOG(WARN, "Fail to open iter, ", K(ret));
}
if (use_fuse_row_cache) {
access_ctx_.use_fuse_row_cache_ = true;
access_ctx_.fuse_row_cache_hit_rate_ = 100L;
}
}
return ret;
@ -274,17 +277,27 @@ int ObSingleRowGetter::get_next_row(ObNewRow *&row)
int ret = OB_SUCCESS;
row = nullptr;
while (OB_SUCC(ret)) {
ObStoreRow *store_row = NULL;
blocksstable::ObDatumRow *store_row = NULL;
if (OB_FAIL(single_merge_->get_next_row(store_row))) {
if (OB_ITER_END != ret) {
STORAGE_LOG(WARN, "failed to get next row", K(ret));
}
} else if (ObActionFlag::OP_ROW_EXIST == store_row->flag_) {
row = &store_row->row_val_;
} else if (store_row->row_flag_.is_exist_without_delete()) {
//TODO remove obnewrow
row = &store_row->get_new_row();
break;
}
}
if (OB_SUCC(ret) || OB_ITER_END == ret) {
// check txn status not aborted, which cause readout incorrect result
if (store_ctx_->mvcc_acc_ctx_.snapshot_.tx_id_.is_valid() &&
store_ctx_->mvcc_acc_ctx_.mem_ctx_ &&
store_ctx_->mvcc_acc_ctx_.mem_ctx_->is_trans_rollbacked()) {
ret = OB_TRANS_KILLED;
STORAGE_LOG(WARN, "txn has terminated", K(ret));
}
}
return ret;
}
} // end namespace storage
} // end namespace oceanbase
} // end namespace storage
} // end namespace oceanbase