remove table schema from complement data task param.
This commit is contained in:
@ -176,9 +176,9 @@ int ObUniqueIndexChecker::scan_table_with_column_checksum(
|
|||||||
LOG_WARN("snapshot version has been discarded", K(ret));
|
LOG_WARN("snapshot version has been discarded", K(ret));
|
||||||
}
|
}
|
||||||
} else if (OB_FAIL(local_scan.init(*param.col_ids_, *param.org_col_ids_, *param.output_projector_,
|
} else if (OB_FAIL(local_scan.init(*param.col_ids_, *param.org_col_ids_, *param.output_projector_,
|
||||||
param.data_table_schema_, param.snapshot_version_, trans_service, param.index_schema_, true/*output org cols only*/))) {
|
*param.data_table_schema_, param.snapshot_version_, trans_service, *param.index_schema_, true/*output org cols only*/))) {
|
||||||
LOG_WARN("init local scan failed", K(ret));
|
LOG_WARN("init local scan failed", K(ret));
|
||||||
} else if (OB_FAIL(local_scan.table_scan(ls_id_, tablet_id_, iterator, query_flag, range, nullptr))) {
|
} else if (OB_FAIL(local_scan.table_scan(*param.data_table_schema_, ls_id_, tablet_id_, iterator, query_flag, range, nullptr))) {
|
||||||
LOG_WARN("fail to table scan", K(ret));
|
LOG_WARN("fail to table scan", K(ret));
|
||||||
} else {
|
} else {
|
||||||
const ObColDescIArray &out_cols = *param.org_col_ids_;
|
const ObColDescIArray &out_cols = *param.org_col_ids_;
|
||||||
|
|||||||
@ -95,12 +95,10 @@ int ObComplementDataParam::init(const ObDDLBuildSingleReplicaRequestArg &arg)
|
|||||||
} else if (OB_ISNULL(data_table_schema)) {
|
} else if (OB_ISNULL(data_table_schema)) {
|
||||||
ret = OB_TABLE_NOT_EXIST;
|
ret = OB_TABLE_NOT_EXIST;
|
||||||
LOG_WARN("data table schema not exist", K(ret), K(arg));
|
LOG_WARN("data table schema not exist", K(ret), K(arg));
|
||||||
} else if (OB_FAIL(deep_copy_table_schemas(data_table_schema, hidden_table_schema))) {
|
|
||||||
LOG_WARN("fail to deep copy table schema", K(ret));
|
|
||||||
} else if (OB_FAIL(guard.switch_to(tenant_id))) {
|
} else if (OB_FAIL(guard.switch_to(tenant_id))) {
|
||||||
LOG_WARN("fail to switch to tenant", K(ret), K(arg));
|
LOG_WARN("fail to switch to tenant", K(ret), K(arg));
|
||||||
} else if (FALSE_IT(snapshot_version_ = arg.snapshot_version_)) {
|
} else if (FALSE_IT(snapshot_version_ = arg.snapshot_version_)) {
|
||||||
} else if (OB_FAIL(split_task_ranges(arg.ls_id_, arg.source_tablet_id_, arg.parallelism_))) {
|
} else if (OB_FAIL(split_task_ranges(arg.ls_id_, arg.source_tablet_id_, data_table_schema->get_tablet_size(), arg.parallelism_))) {
|
||||||
LOG_WARN("fail to init concurrent params", K(ret), K(arg));
|
LOG_WARN("fail to init concurrent params", K(ret), K(arg));
|
||||||
} else if (OB_FAIL(ObCompatModeGetter::get_table_compat_mode(tenant_id, arg.source_table_id_, compat_mode_))) {
|
} else if (OB_FAIL(ObCompatModeGetter::get_table_compat_mode(tenant_id, arg.source_table_id_, compat_mode_))) {
|
||||||
LOG_WARN("failed to get compat mode", K(ret), K(arg));
|
LOG_WARN("failed to get compat mode", K(ret), K(arg));
|
||||||
@ -108,6 +106,8 @@ int ObComplementDataParam::init(const ObDDLBuildSingleReplicaRequestArg &arg)
|
|||||||
is_inited_ = true;
|
is_inited_ = true;
|
||||||
tenant_id_ = tenant_id;
|
tenant_id_ = tenant_id;
|
||||||
ls_id_ = arg.ls_id_;
|
ls_id_ = arg.ls_id_;
|
||||||
|
source_table_id_ = arg.source_table_id_;
|
||||||
|
dest_table_id_ = arg.dest_schema_id_;
|
||||||
source_tablet_id_ = arg.source_tablet_id_;
|
source_tablet_id_ = arg.source_tablet_id_;
|
||||||
dest_tablet_id_ = arg.dest_tablet_id_;
|
dest_tablet_id_ = arg.dest_tablet_id_;
|
||||||
schema_version_ = arg.schema_version_;
|
schema_version_ = arg.schema_version_;
|
||||||
@ -122,66 +122,11 @@ int ObComplementDataParam::init(const ObDDLBuildSingleReplicaRequestArg &arg)
|
|||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
int ObComplementDataParam::deep_copy_table_schemas(const ObTableSchema *data_table_schema, const ObTableSchema *hidden_table_schema)
|
|
||||||
{
|
|
||||||
int ret = OB_SUCCESS;
|
|
||||||
if (OB_UNLIKELY(is_inited_)) {
|
|
||||||
ret = OB_INIT_TWICE;
|
|
||||||
LOG_WARN("ObComplementDataParam has been inited before", K(ret));
|
|
||||||
} else if (OB_ISNULL(data_table_schema) || OB_ISNULL(hidden_table_schema)) {
|
|
||||||
ret = OB_INVALID_ARGUMENT;
|
|
||||||
LOG_WARN("invalid arguments", K(ret), KP(data_table_schema), KP(hidden_table_schema));
|
|
||||||
} else {
|
|
||||||
ObIAllocator &allocator = allocator_;
|
|
||||||
const int64_t alloc_size = sizeof(ObTableSchema);
|
|
||||||
char *buf_for_data_schema = nullptr;
|
|
||||||
char *buf_for_hidden_schema = nullptr;
|
|
||||||
if (OB_ISNULL(buf_for_data_schema = static_cast<char *>(allocator.alloc(alloc_size)))) {
|
|
||||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
||||||
LOG_WARN("alloc memory failed", K(ret));
|
|
||||||
} else if (OB_ISNULL(buf_for_hidden_schema = static_cast<char *>(allocator.alloc(alloc_size)))) {
|
|
||||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
|
||||||
LOG_WARN("alloc memory failed", K(ret));
|
|
||||||
} else {
|
|
||||||
ObTableSchema *deep_copy_data_table_schema = nullptr;
|
|
||||||
ObTableSchema *deep_copy_hidden_table_schema = nullptr;
|
|
||||||
deep_copy_data_table_schema = new (buf_for_data_schema) ObTableSchema(&allocator);
|
|
||||||
deep_copy_hidden_table_schema = new (buf_for_hidden_schema) ObTableSchema(&allocator);
|
|
||||||
if (OB_FAIL(deep_copy_data_table_schema->assign(*data_table_schema))) {
|
|
||||||
LOG_WARN("fail to assign data table schema", K(ret));
|
|
||||||
} else if (OB_FAIL(deep_copy_hidden_table_schema->assign(*hidden_table_schema))) {
|
|
||||||
LOG_WARN("fail to assign hidden table schema", K(ret));
|
|
||||||
} else {
|
|
||||||
data_table_schema_ = deep_copy_data_table_schema;
|
|
||||||
hidden_table_schema_ = deep_copy_hidden_table_schema;
|
|
||||||
}
|
|
||||||
if (OB_FAIL(ret)) {
|
|
||||||
if (nullptr != deep_copy_data_table_schema) {
|
|
||||||
deep_copy_data_table_schema->~ObTableSchema();
|
|
||||||
deep_copy_data_table_schema = nullptr;
|
|
||||||
}
|
|
||||||
if (nullptr != buf_for_data_schema) {
|
|
||||||
allocator_.free(buf_for_data_schema);
|
|
||||||
buf_for_data_schema = nullptr;
|
|
||||||
}
|
|
||||||
if (nullptr != deep_copy_hidden_table_schema) {
|
|
||||||
deep_copy_hidden_table_schema->~ObTableSchema();
|
|
||||||
deep_copy_hidden_table_schema = nullptr;
|
|
||||||
}
|
|
||||||
if (nullptr != buf_for_hidden_schema) {
|
|
||||||
allocator_.free(buf_for_hidden_schema);
|
|
||||||
buf_for_hidden_schema = nullptr;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return ret;
|
|
||||||
}
|
|
||||||
|
|
||||||
// split task ranges to do table scan based on the whole range on the specified tablet.
|
// split task ranges to do table scan based on the whole range on the specified tablet.
|
||||||
int ObComplementDataParam::split_task_ranges(
|
int ObComplementDataParam::split_task_ranges(
|
||||||
const share::ObLSID &ls_id,
|
const share::ObLSID &ls_id,
|
||||||
const common::ObTabletID &tablet_id,
|
const common::ObTabletID &tablet_id,
|
||||||
|
const int64_t tablet_size,
|
||||||
const int64_t hint_parallelism)
|
const int64_t hint_parallelism)
|
||||||
{
|
{
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
@ -213,7 +158,7 @@ int ObComplementDataParam::split_task_ranges(
|
|||||||
ObArrayArray<ObStoreRange> multi_range_split_array;
|
ObArrayArray<ObStoreRange> multi_range_split_array;
|
||||||
ObParallelBlockRangeTaskParams params;
|
ObParallelBlockRangeTaskParams params;
|
||||||
params.parallelism_ = hint_parallelism;
|
params.parallelism_ = hint_parallelism;
|
||||||
params.expected_task_load_ = data_table_schema_->get_tablet_size() / 1024 / 1024;
|
params.expected_task_load_ = tablet_size / 1024 / 1024;
|
||||||
if (OB_FAIL(ranges.push_back(range))) {
|
if (OB_FAIL(ranges.push_back(range))) {
|
||||||
LOG_WARN("push back range failed", K(ret));
|
LOG_WARN("push back range failed", K(ret));
|
||||||
} else if (OB_FAIL(tablet_service->get_multi_ranges_cost(tablet_id,
|
} else if (OB_FAIL(tablet_service->get_multi_ranges_cost(tablet_id,
|
||||||
@ -439,13 +384,24 @@ int ObComplementDataDag::prepare_context()
|
|||||||
{
|
{
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
ObDataStoreDesc data_desc;
|
ObDataStoreDesc data_desc;
|
||||||
|
ObSchemaGetterGuard schema_guard;
|
||||||
|
const ObTableSchema *hidden_table_schema = nullptr;
|
||||||
if (OB_UNLIKELY(!is_inited_)) {
|
if (OB_UNLIKELY(!is_inited_)) {
|
||||||
ret = OB_NOT_INIT;
|
ret = OB_NOT_INIT;
|
||||||
LOG_WARN("ObComplementDataDag not init", K(ret));
|
LOG_WARN("ObComplementDataDag not init", K(ret));
|
||||||
} else if (OB_UNLIKELY(!param_.is_valid())) {
|
} else if (OB_UNLIKELY(!param_.is_valid())) {
|
||||||
ret = OB_ERR_UNEXPECTED;
|
ret = OB_ERR_UNEXPECTED;
|
||||||
LOG_WARN("error unexpected", K(ret), K(param_));
|
LOG_WARN("error unexpected", K(ret), K(param_));
|
||||||
} else if (OB_FAIL(data_desc.init(*param_.hidden_table_schema_,
|
} else if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard(
|
||||||
|
param_.tenant_id_, schema_guard, param_.schema_version_))) {
|
||||||
|
LOG_WARN("fail to get tenant schema guard", K(ret), K(param_));
|
||||||
|
} else if (OB_FAIL(schema_guard.get_table_schema(param_.tenant_id_,
|
||||||
|
param_.dest_table_id_, hidden_table_schema))) {
|
||||||
|
LOG_WARN("fail to get hidden table schema", K(ret), K(param_));
|
||||||
|
} else if (OB_ISNULL(hidden_table_schema)) {
|
||||||
|
ret = OB_TABLE_NOT_EXIST;
|
||||||
|
LOG_WARN("hidden table schema not exist", K(ret), K(param_));
|
||||||
|
} else if (OB_FAIL(data_desc.init(*hidden_table_schema,
|
||||||
param_.ls_id_,
|
param_.ls_id_,
|
||||||
param_.dest_tablet_id_,
|
param_.dest_tablet_id_,
|
||||||
MAJOR_MERGE,
|
MAJOR_MERGE,
|
||||||
@ -459,7 +415,7 @@ int ObComplementDataDag::prepare_context()
|
|||||||
data_desc.is_ddl_ = true;
|
data_desc.is_ddl_ = true;
|
||||||
if (OB_FAIL(data_desc.col_desc_array_.init(data_desc.row_column_count_))) {
|
if (OB_FAIL(data_desc.col_desc_array_.init(data_desc.row_column_count_))) {
|
||||||
LOG_WARN("failed to reserve column desc array", K(ret));
|
LOG_WARN("failed to reserve column desc array", K(ret));
|
||||||
} else if (OB_FAIL(param_.hidden_table_schema_->get_rowkey_column_ids(data_desc.col_desc_array_))) {
|
} else if (OB_FAIL(hidden_table_schema->get_rowkey_column_ids(data_desc.col_desc_array_))) {
|
||||||
LOG_WARN("failed to get rowkey column ids", K(ret));
|
LOG_WARN("failed to get rowkey column ids", K(ret));
|
||||||
} else if (OB_FAIL(storage::ObMultiVersionRowkeyHelpper::add_extra_rowkey_cols(data_desc.col_desc_array_))) {
|
} else if (OB_FAIL(storage::ObMultiVersionRowkeyHelpper::add_extra_rowkey_cols(data_desc.col_desc_array_))) {
|
||||||
LOG_WARN("failed to add extra rowkey cols", K(ret));
|
LOG_WARN("failed to add extra rowkey cols", K(ret));
|
||||||
@ -494,7 +450,7 @@ int64_t ObComplementDataDag::hash() const
|
|||||||
LOG_ERROR("table schema must not be NULL", K(tmp_ret), K(is_inited_), K(param_));
|
LOG_ERROR("table schema must not be NULL", K(tmp_ret), K(is_inited_), K(param_));
|
||||||
} else {
|
} else {
|
||||||
hash_val = param_.tenant_id_ + param_.ls_id_.hash()
|
hash_val = param_.tenant_id_ + param_.ls_id_.hash()
|
||||||
+ param_.data_table_schema_->get_table_id() + param_.hidden_table_schema_->get_table_id()
|
+ param_.source_table_id_ + param_.dest_table_id_
|
||||||
+ param_.source_tablet_id_.hash() + param_.dest_tablet_id_.hash() + ObDagType::DAG_TYPE_DDL;
|
+ param_.source_tablet_id_.hash() + param_.dest_tablet_id_.hash() + ObDagType::DAG_TYPE_DDL;
|
||||||
}
|
}
|
||||||
return hash_val;
|
return hash_val;
|
||||||
@ -513,8 +469,7 @@ bool ObComplementDataDag::operator==(const ObIDag &other) const
|
|||||||
LOG_ERROR("invalid argument", K(tmp_ret), K(param_), K(dag.param_));
|
LOG_ERROR("invalid argument", K(tmp_ret), K(param_), K(dag.param_));
|
||||||
} else {
|
} else {
|
||||||
is_equal = (param_.tenant_id_ == dag.param_.tenant_id_) && (param_.ls_id_ == dag.param_.ls_id_) &&
|
is_equal = (param_.tenant_id_ == dag.param_.tenant_id_) && (param_.ls_id_ == dag.param_.ls_id_) &&
|
||||||
(param_.data_table_schema_->get_table_id() == dag.param_.data_table_schema_->get_table_id()) &&
|
(param_.source_table_id_ == dag.param_.source_table_id_) && (param_.dest_table_id_ == dag.param_.dest_table_id_) &&
|
||||||
(param_.hidden_table_schema_->get_table_id() == dag.param_.hidden_table_schema_->get_table_id()) &&
|
|
||||||
(param_.source_tablet_id_ == dag.param_.source_tablet_id_) && (param_.dest_tablet_id_ == dag.param_.dest_tablet_id_);
|
(param_.source_tablet_id_ == dag.param_.source_tablet_id_) && (param_.dest_tablet_id_ == dag.param_.dest_tablet_id_);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -544,8 +499,8 @@ int ObComplementDataDag::report_replica_build_status()
|
|||||||
arg.tenant_id_ = param_.tenant_id_;
|
arg.tenant_id_ = param_.tenant_id_;
|
||||||
arg.ls_id_ = param_.ls_id_;
|
arg.ls_id_ = param_.ls_id_;
|
||||||
arg.tablet_id_ = param_.source_tablet_id_;
|
arg.tablet_id_ = param_.source_tablet_id_;
|
||||||
arg.source_table_id_ = param_.data_table_schema_->get_table_id();
|
arg.source_table_id_ = param_.source_table_id_;
|
||||||
arg.dest_schema_id_ = param_.hidden_table_schema_->get_table_id();
|
arg.dest_schema_id_ = param_.dest_table_id_;
|
||||||
arg.ret_code_ = context_.complement_data_ret_;
|
arg.ret_code_ = context_.complement_data_ret_;
|
||||||
arg.snapshot_version_ = param_.snapshot_version_;
|
arg.snapshot_version_ = param_.snapshot_version_;
|
||||||
arg.schema_version_ = param_.schema_version_;
|
arg.schema_version_ = param_.schema_version_;
|
||||||
@ -578,7 +533,7 @@ int ObComplementDataDag::fill_comment(char *buf, const int64_t buf_len) const
|
|||||||
ret = OB_INVALID_ARGUMENT;
|
ret = OB_INVALID_ARGUMENT;
|
||||||
LOG_WARN("invalid param", K(ret), K(param_));
|
LOG_WARN("invalid param", K(ret), K(param_));
|
||||||
} else if (OB_FAIL(databuff_printf(buf, buf_len, "complement data task: logstream_id=%ld, source_tablet_id=%ld, dest_tablet_id=%ld, data_table_id=%ld, target_table_id=%ld, schema_version=%ld, snapshot_version=%ld",
|
} else if (OB_FAIL(databuff_printf(buf, buf_len, "complement data task: logstream_id=%ld, source_tablet_id=%ld, dest_tablet_id=%ld, data_table_id=%ld, target_table_id=%ld, schema_version=%ld, snapshot_version=%ld",
|
||||||
param_.ls_id_.id(), param_.source_tablet_id_.id(), param_.dest_tablet_id_.id(), param_.data_table_schema_->get_table_id(), param_.hidden_table_schema_->get_table_id(), param_.schema_version_, param_.snapshot_version_))) {
|
param_.ls_id_.id(), param_.source_tablet_id_.id(), param_.dest_tablet_id_.id(), param_.source_table_id_, param_.dest_table_id_, param_.schema_version_, param_.snapshot_version_))) {
|
||||||
LOG_WARN("fail to fill comment", K(ret), K(param_));
|
LOG_WARN("fail to fill comment", K(ret), K(param_));
|
||||||
}
|
}
|
||||||
return ret;
|
return ret;
|
||||||
@ -673,15 +628,26 @@ int ObComplementWriteTask::init(const int64_t task_id, ObComplementDataParam &pa
|
|||||||
ObComplementDataContext &context)
|
ObComplementDataContext &context)
|
||||||
{
|
{
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
|
ObSchemaGetterGuard schema_guard;
|
||||||
|
const ObTableSchema *hidden_table_schema = nullptr;
|
||||||
if (OB_UNLIKELY(is_inited_)) {
|
if (OB_UNLIKELY(is_inited_)) {
|
||||||
ret = OB_INIT_TWICE;
|
ret = OB_INIT_TWICE;
|
||||||
LOG_WARN("ObComplementWriteTask has already been inited", K(ret));
|
LOG_WARN("ObComplementWriteTask has already been inited", K(ret));
|
||||||
} else if (task_id < 0 || !param.is_valid()) {
|
} else if (task_id < 0 || !param.is_valid()) {
|
||||||
ret = OB_INVALID_ARGUMENT;
|
ret = OB_INVALID_ARGUMENT;
|
||||||
LOG_WARN("invalid arguments", K(ret), K(task_id), K(param), K(context));
|
LOG_WARN("invalid arguments", K(ret), K(task_id), K(param), K(context));
|
||||||
|
} else if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard(
|
||||||
|
param.tenant_id_, schema_guard, param.schema_version_))) {
|
||||||
|
LOG_WARN("fail to get tenant schema guard", K(ret), K(param));
|
||||||
|
} else if (OB_FAIL(schema_guard.get_table_schema(param.tenant_id_,
|
||||||
|
param.dest_table_id_, hidden_table_schema))) {
|
||||||
|
LOG_WARN("fail to get hidden table schema", K(ret), K(param));
|
||||||
|
} else if (OB_ISNULL(hidden_table_schema)) {
|
||||||
|
ret = OB_TABLE_NOT_EXIST;
|
||||||
|
LOG_WARN("hidden table schema not exist", K(ret), K(param));
|
||||||
} else if (OB_FAIL(write_row_.init(
|
} else if (OB_FAIL(write_row_.init(
|
||||||
param.allocator_, param.hidden_table_schema_->get_column_count() + storage::ObMultiVersionRowkeyHelpper::get_extra_rowkey_col_cnt()))) {
|
param.allocator_, hidden_table_schema->get_column_count() + storage::ObMultiVersionRowkeyHelpper::get_extra_rowkey_col_cnt()))) {
|
||||||
LOG_WARN("Fail to init write row", K(ret), K(param.hidden_table_schema_->get_column_count()));
|
LOG_WARN("Fail to init write row", K(ret));
|
||||||
} else {
|
} else {
|
||||||
write_row_.row_flag_.set_flag(ObDmlFlag::DF_INSERT);
|
write_row_.row_flag_.set_flag(ObDmlFlag::DF_INSERT);
|
||||||
task_id_ = task_id;
|
task_id_ = task_id;
|
||||||
@ -747,7 +713,7 @@ int ObComplementWriteTask::generate_next_task(ObITask *&next_task)
|
|||||||
LOG_WARN("fail to init complement write task", K(ret));
|
LOG_WARN("fail to init complement write task", K(ret));
|
||||||
} else {
|
} else {
|
||||||
next_task = write_task;
|
next_task = write_task;
|
||||||
LOG_INFO("generate next complement write task", K(ret), "hidden_table_id", param_->hidden_table_schema_->get_table_id());
|
LOG_INFO("generate next complement write task", K(ret), K(param_->dest_table_id_));
|
||||||
}
|
}
|
||||||
if (OB_FAIL(ret) && OB_NOT_NULL(context_)) {
|
if (OB_FAIL(ret) && OB_NOT_NULL(context_)) {
|
||||||
if (OB_ITER_END != ret) {
|
if (OB_ITER_END != ret) {
|
||||||
@ -758,23 +724,36 @@ int ObComplementWriteTask::generate_next_task(ObITask *&next_task)
|
|||||||
}
|
}
|
||||||
|
|
||||||
//generate col_ids and projector based on table_schema
|
//generate col_ids and projector based on table_schema
|
||||||
int ObComplementWriteTask::generate_col_param(
|
int ObComplementWriteTask::generate_col_param()
|
||||||
const ObTableSchema *data_table_schema,
|
|
||||||
const ObTableSchema *hidden_table_schema)
|
|
||||||
{
|
{
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
col_ids_.reuse();
|
col_ids_.reuse();
|
||||||
org_col_ids_.reuse();
|
org_col_ids_.reuse();
|
||||||
output_projector_.reuse();
|
output_projector_.reuse();
|
||||||
ObArray<ObColDesc> tmp_col_ids;
|
ObArray<ObColDesc> tmp_col_ids;
|
||||||
if (OB_ISNULL(data_table_schema) || OB_ISNULL(hidden_table_schema)) {
|
ObSchemaGetterGuard schema_guard;
|
||||||
ret = OB_INVALID_ARGUMENT;
|
const ObTableSchema *data_table_schema = nullptr;
|
||||||
LOG_WARN("invalid arguments", K(ret), KP(data_table_schema), KP(hidden_table_schema));
|
const ObTableSchema *hidden_table_schema = nullptr;
|
||||||
} else if (OB_UNLIKELY(!data_table_schema->is_valid() || !hidden_table_schema->is_valid())) {
|
if (OB_UNLIKELY(!is_inited_)) {
|
||||||
ret = OB_INVALID_ARGUMENT;
|
ret = OB_NOT_INIT;
|
||||||
LOG_WARN("invalid arguments", K(ret), K(*data_table_schema), K(*hidden_table_schema));
|
LOG_WARN("not init", K(ret));
|
||||||
|
} else if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard(
|
||||||
|
param_->tenant_id_, schema_guard, param_->schema_version_))) {
|
||||||
|
LOG_WARN("fail to get tenant schema guard", K(ret), KPC(param_));
|
||||||
|
} else if (OB_FAIL(schema_guard.get_table_schema(param_->tenant_id_,
|
||||||
|
param_->source_table_id_, data_table_schema))) {
|
||||||
|
LOG_WARN("fail to get data table schema", K(ret), K(arg));
|
||||||
|
} else if (OB_ISNULL(data_table_schema)) {
|
||||||
|
ret = OB_TABLE_NOT_EXIST;
|
||||||
|
LOG_WARN("data table schema not exist", K(ret), K(arg));
|
||||||
|
} else if (OB_FAIL(schema_guard.get_table_schema(param_->tenant_id_,
|
||||||
|
param_->dest_table_id_, hidden_table_schema))) {
|
||||||
|
LOG_WARN("fail to get hidden table schema", K(ret), KPC(param_));
|
||||||
|
} else if (OB_ISNULL(hidden_table_schema)) {
|
||||||
|
ret = OB_TABLE_NOT_EXIST;
|
||||||
|
LOG_WARN("hidden table schema not exist", K(ret), KPC(param_));
|
||||||
} else if (OB_FAIL(hidden_table_schema->get_store_column_ids(tmp_col_ids, false))) {
|
} else if (OB_FAIL(hidden_table_schema->get_store_column_ids(tmp_col_ids, false))) {
|
||||||
LOG_WARN("fail to get column ids", K(ret), K(*hidden_table_schema));
|
LOG_WARN("fail to get column ids", K(ret), KPC(hidden_table_schema));
|
||||||
} else if (OB_FAIL(org_col_ids_.assign(tmp_col_ids))) {
|
} else if (OB_FAIL(org_col_ids_.assign(tmp_col_ids))) {
|
||||||
LOG_WARN("fail to assign col descs", K(ret), K(tmp_col_ids));
|
LOG_WARN("fail to assign col descs", K(ret), K(tmp_col_ids));
|
||||||
} else {
|
} else {
|
||||||
@ -832,24 +811,17 @@ int ObComplementWriteTask::local_scan_by_range()
|
|||||||
{
|
{
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
int64_t start_time = ObTimeUtility::current_time();
|
int64_t start_time = ObTimeUtility::current_time();
|
||||||
uint64_t tenant_id = OB_INVALID_ID;
|
|
||||||
uint64_t hidden_table_id = OB_INVALID_ID;
|
|
||||||
int64_t concurrent_cnt = 0;
|
int64_t concurrent_cnt = 0;
|
||||||
const ObTableSchema *data_table_schema = nullptr;
|
|
||||||
const ObTableSchema *hidden_table_schema = nullptr;
|
|
||||||
if (OB_ISNULL(param_) || OB_UNLIKELY(!param_->is_valid())) {
|
if (OB_ISNULL(param_) || OB_UNLIKELY(!param_->is_valid())) {
|
||||||
ret = OB_INVALID_ARGUMENT;
|
ret = OB_INVALID_ARGUMENT;
|
||||||
LOG_WARN("invalid arguments", K(ret), K(idx), KP(param_));
|
LOG_WARN("invalid arguments", K(ret), K(idx), KP(param_));
|
||||||
} else {
|
} else {
|
||||||
data_table_schema = param_->data_table_schema_;
|
|
||||||
hidden_table_schema = param_->hidden_table_schema_;
|
|
||||||
tenant_id = param_->tenant_id_;
|
|
||||||
concurrent_cnt = param_->concurrent_cnt_;
|
concurrent_cnt = param_->concurrent_cnt_;
|
||||||
LOG_INFO("start to do local scan by range", "data_table_id", data_table_schema->get_table_id(), K(task_id_), K(concurrent_cnt));
|
LOG_INFO("start to do local scan by range", K(task_id_), K(concurrent_cnt), KPC(param_));
|
||||||
}
|
}
|
||||||
if (OB_FAIL(ret)) {
|
if (OB_FAIL(ret)) {
|
||||||
// do nothing
|
// do nothing
|
||||||
} else if (OB_FAIL(generate_col_param(data_table_schema, hidden_table_schema))) {
|
} else if (OB_FAIL(generate_col_param())) {
|
||||||
LOG_WARN("fail to get column ids", K(ret));
|
LOG_WARN("fail to get column ids", K(ret));
|
||||||
} else if (OB_FAIL(do_local_scan())) {
|
} else if (OB_FAIL(do_local_scan())) {
|
||||||
LOG_WARN("fail to do local scan", K(ret), K_(col_ids), K_(org_col_ids));
|
LOG_WARN("fail to do local scan", K(ret), K_(col_ids), K_(org_col_ids));
|
||||||
@ -881,7 +853,6 @@ int ObComplementWriteTask::do_local_scan()
|
|||||||
ObSSTable *sstable = nullptr;
|
ObSSTable *sstable = nullptr;
|
||||||
ObTransService *trans_service = nullptr;
|
ObTransService *trans_service = nullptr;
|
||||||
ObSEArray<ObITable *, MAX_SSTABLE_CNT_IN_STORAGE> sstables;
|
ObSEArray<ObITable *, MAX_SSTABLE_CNT_IN_STORAGE> sstables;
|
||||||
const uint64_t data_table_id = param_->data_table_schema_->get_table_id();
|
|
||||||
const uint64_t tenant_id = param_->tenant_id_;
|
const uint64_t tenant_id = param_->tenant_id_;
|
||||||
ObTxDesc *read_tx_desc = nullptr; // for reading lob column from aux_lob_table by table_scan
|
ObTxDesc *read_tx_desc = nullptr; // for reading lob column from aux_lob_table by table_scan
|
||||||
|
|
||||||
@ -904,22 +875,49 @@ int ObComplementWriteTask::do_local_scan()
|
|||||||
LOG_WARN("fail to get range", K(ret));
|
LOG_WARN("fail to get range", K(ret));
|
||||||
} else if (OB_FAIL(datum_range.from_range(range, allocator))) {
|
} else if (OB_FAIL(datum_range.from_range(range, allocator))) {
|
||||||
STORAGE_LOG(WARN, "Failed to transfer datum range", K(ret), K(range));
|
STORAGE_LOG(WARN, "Failed to transfer datum range", K(ret), K(range));
|
||||||
} else if (OB_FAIL(local_scan.init(col_ids_,
|
} else {
|
||||||
org_col_ids_,
|
ObSchemaGetterGuard schema_guard;
|
||||||
output_projector_,
|
const ObTableSchema *data_table_schema = nullptr;
|
||||||
param_->data_table_schema_,
|
const ObTableSchema *hidden_table_schema = nullptr;
|
||||||
param_->snapshot_version_,
|
if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard(
|
||||||
trans_service,
|
tenant_id, schema_guard, param_->schema_version_))) {
|
||||||
param_->hidden_table_schema_,
|
LOG_WARN("fail to get tenant schema guard", K(ret), KPC(param_));
|
||||||
false/*output all columns of hidden table*/))) {
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
||||||
LOG_WARN("fail to init local scan param", K(ret), K(*param_));
|
param_->source_table_id_, data_table_schema))) {
|
||||||
} else if (OB_FAIL(ObInsertLobColumnHelper::start_trans(
|
LOG_WARN("fail to get data table schema", K(ret), K(arg));
|
||||||
param_->ls_id_, true/*is_for_read*/, INT64_MAX, read_tx_desc))) {
|
} else if (OB_ISNULL(data_table_schema)) {
|
||||||
LOG_WARN("fail to get tx_desc", K(ret));
|
ret = OB_TABLE_NOT_EXIST;
|
||||||
} else if (OB_FAIL(local_scan.table_scan(param_->ls_id_, param_->source_tablet_id_, iterator, query_flag, datum_range, read_tx_desc))) {
|
LOG_WARN("data table schema not exist", K(ret), K(arg));
|
||||||
LOG_WARN("fail to do table scan", K(ret));
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id,
|
||||||
} else if (OB_FAIL(append_row(local_scan))) {
|
param_->dest_table_id_, hidden_table_schema))) {
|
||||||
LOG_WARN("fail to append row", K(ret));
|
LOG_WARN("fail to get hidden table schema", K(ret), KPC(param_));
|
||||||
|
} else if (OB_ISNULL(hidden_table_schema)) {
|
||||||
|
ret = OB_TABLE_NOT_EXIST;
|
||||||
|
LOG_WARN("hidden table schema not exist", K(ret), KPC(param_));
|
||||||
|
} else if (OB_FAIL(local_scan.init(col_ids_,
|
||||||
|
org_col_ids_,
|
||||||
|
output_projector_,
|
||||||
|
*data_table_schema,
|
||||||
|
param_->snapshot_version_,
|
||||||
|
trans_service,
|
||||||
|
*hidden_table_schema,
|
||||||
|
false/*output all columns of hidden table*/))) {
|
||||||
|
LOG_WARN("fail to init local scan param", K(ret), K(*param_));
|
||||||
|
} else if (OB_FAIL(ObInsertLobColumnHelper::start_trans(
|
||||||
|
param_->ls_id_, true/*is_for_read*/, INT64_MAX, read_tx_desc))) {
|
||||||
|
LOG_WARN("fail to get tx_desc", K(ret));
|
||||||
|
} else if (OB_FAIL(local_scan.table_scan(*data_table_schema,
|
||||||
|
param_->ls_id_,
|
||||||
|
param_->source_tablet_id_,
|
||||||
|
iterator,
|
||||||
|
query_flag,
|
||||||
|
datum_range, read_tx_desc))) {
|
||||||
|
LOG_WARN("fail to do table scan", K(ret));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (FAILEDx(append_row(local_scan))) {
|
||||||
|
LOG_WARN("append row failed", K(ret));
|
||||||
}
|
}
|
||||||
|
|
||||||
const int64_t timeout_ts = ObTimeUtility::current_time() + 3000000; // 3s
|
const int64_t timeout_ts = ObTimeUtility::current_time() + 3000000; // 3s
|
||||||
@ -934,10 +932,10 @@ int ObComplementWriteTask::do_local_scan()
|
|||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
int ObComplementWriteTask::add_extra_rowkey(const int64_t extra_rowkey_cnt, const blocksstable::ObDatumRow &row)
|
int ObComplementWriteTask::add_extra_rowkey(const int64_t rowkey_cnt, const int64_t extra_rowkey_cnt, const blocksstable::ObDatumRow &row)
|
||||||
{
|
{
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
int64_t rowkey_column_count = param_->hidden_table_schema_->get_rowkey_column_num();
|
int64_t rowkey_column_count = rowkey_cnt;
|
||||||
if (OB_UNLIKELY(write_row_.get_capacity() < row.count_ + extra_rowkey_cnt ||
|
if (OB_UNLIKELY(write_row_.get_capacity() < row.count_ + extra_rowkey_cnt ||
|
||||||
row.count_ < rowkey_column_count)) {
|
row.count_ < rowkey_column_count)) {
|
||||||
ret = OB_ERR_UNEXPECTED;
|
ret = OB_ERR_UNEXPECTED;
|
||||||
@ -975,70 +973,87 @@ int ObComplementWriteTask::append_row(ObLocalScan &local_scan)
|
|||||||
int64_t t3 = 0;
|
int64_t t3 = 0;
|
||||||
int64_t lob_cnt = 0;
|
int64_t lob_cnt = 0;
|
||||||
ObArenaAllocator lob_allocator(ObModIds::OB_LOB_ACCESS_BUFFER, OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID());
|
ObArenaAllocator lob_allocator(ObModIds::OB_LOB_ACCESS_BUFFER, OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID());
|
||||||
|
ObStoreRow reshaped_row;
|
||||||
|
reshaped_row.flag_.set_flag(ObDmlFlag::DF_INSERT);
|
||||||
|
ObArenaAllocator allocator(lib::ObLabel("CompDataTaskTmp"));
|
||||||
|
ObRowReshape *reshape_ptr = nullptr;
|
||||||
|
ObSQLMode sql_mode_for_ddl_reshape = SMO_TRADITIONAL;
|
||||||
|
ObDatumRow datum_row;
|
||||||
|
int64_t rowkey_column_cnt = 0;
|
||||||
|
const int64_t extra_rowkey_cnt = storage::ObMultiVersionRowkeyHelpper::get_extra_rowkey_col_cnt();
|
||||||
if (OB_UNLIKELY(!is_inited_)) {
|
if (OB_UNLIKELY(!is_inited_)) {
|
||||||
ret = OB_NOT_INIT;
|
ret = OB_NOT_INIT;
|
||||||
LOG_WARN("ObComplementWriteTask is not inited", K(ret));
|
LOG_WARN("ObComplementWriteTask is not inited", K(ret));
|
||||||
} else if (OB_ISNULL(param_) || OB_UNLIKELY(!param_->is_valid()) || OB_ISNULL(context_)) {
|
} else if (OB_ISNULL(param_) || OB_UNLIKELY(!param_->is_valid()) || OB_ISNULL(context_)) {
|
||||||
ret = OB_INVALID_ARGUMENT;
|
ret = OB_INVALID_ARGUMENT;
|
||||||
LOG_WARN("invalid arguments", K(param_), KP(context_), K(ret));
|
LOG_WARN("invalid arguments", K(param_), KP(context_), K(ret));
|
||||||
} else if (OB_FAIL(data_desc.init(*param_->hidden_table_schema_,
|
} else {
|
||||||
param_->ls_id_,
|
ObSchemaGetterGuard schema_guard;
|
||||||
param_->dest_tablet_id_,
|
const ObTableSchema *hidden_table_schema = nullptr;
|
||||||
MAJOR_MERGE,
|
if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard(
|
||||||
param_->snapshot_version_,
|
param_->tenant_id_, schema_guard, param_->schema_version_))) {
|
||||||
param_->cluster_version_))) {
|
LOG_WARN("fail to get tenant schema guard", K(ret), KPC(param_));
|
||||||
LOG_WARN("fail to init data store desc", K(ret), K(*param_), K(param_->dest_tablet_id_));
|
} else if (OB_FAIL(schema_guard.get_table_schema(param_->tenant_id_,
|
||||||
} else if (FALSE_IT(data_desc.sstable_index_builder_ = context_->index_builder_)) {
|
param_->dest_table_id_, hidden_table_schema))) {
|
||||||
} else if (FALSE_IT(data_desc.is_ddl_ = true)) {
|
LOG_WARN("fail to get hidden table schema", K(ret), KPC(param_));
|
||||||
} else if (OB_FAIL(param_->get_hidden_table_key(hidden_table_key))) {
|
} else if (OB_ISNULL(hidden_table_schema)) {
|
||||||
LOG_WARN("fail to get hidden table key", K(ret), K(*param_));
|
ret = OB_TABLE_NOT_EXIST;
|
||||||
} else if (OB_UNLIKELY(!hidden_table_key.is_valid())) {
|
LOG_WARN("table not exist", K(ret), K(param_->tenant_id_), K(param_->dest_table_id_));
|
||||||
ret = OB_ERR_UNEXPECTED;
|
} else if (OB_FAIL(data_desc.init(*hidden_table_schema,
|
||||||
LOG_WARN("hidden table key is invalid", K(ret), K(hidden_table_key));
|
param_->ls_id_,
|
||||||
} else if (OB_FAIL(sstable_redo_writer.init(param_->ls_id_, param_->dest_tablet_id_))) {
|
param_->dest_tablet_id_,
|
||||||
LOG_WARN("fail to init sstable redo writer", K(ret));
|
MAJOR_MERGE,
|
||||||
} else if (OB_UNLIKELY(nullptr == static_cast<ObComplementDataDag *>(get_dag()))) {
|
param_->snapshot_version_,
|
||||||
ret = OB_ERR_UNEXPECTED;
|
param_->cluster_version_))) {
|
||||||
LOG_WARN("the dag of this task is null", K(ret));
|
LOG_WARN("fail to init data store desc", K(ret), K(*param_), K(param_->dest_tablet_id_));
|
||||||
} else if (FALSE_IT(sstable_redo_writer.set_start_scn(
|
} else if (FALSE_IT(data_desc.sstable_index_builder_ = context_->index_builder_)) {
|
||||||
static_cast<ObComplementDataDag *>(get_dag())->get_context().data_sstable_redo_writer_.get_start_scn()))) {
|
} else if (FALSE_IT(data_desc.is_ddl_ = true)) {
|
||||||
} else if (OB_FAIL(callback.init(DDL_MB_DATA_TYPE, hidden_table_key, &sstable_redo_writer))) {
|
} else if (OB_FAIL(param_->get_hidden_table_key(hidden_table_key))) {
|
||||||
LOG_WARN("fail to init data callback", K(ret), K(hidden_table_key));
|
LOG_WARN("fail to get hidden table key", K(ret), K(*param_));
|
||||||
} else if (OB_FAIL(writer.open(data_desc, macro_start_seq, &callback))) {
|
} else if (OB_UNLIKELY(!hidden_table_key.is_valid())) {
|
||||||
LOG_WARN("fail to open macro block writer", K(ret), K(data_desc));
|
ret = OB_ERR_UNEXPECTED;
|
||||||
}
|
LOG_WARN("hidden table key is invalid", K(ret), K(hidden_table_key));
|
||||||
int64_t extra_rowkey_cnt = storage::ObMultiVersionRowkeyHelpper::get_extra_rowkey_col_cnt();
|
} else if (OB_FAIL(sstable_redo_writer.init(param_->ls_id_, param_->dest_tablet_id_))) {
|
||||||
ObStoreRow reshaped_row;
|
LOG_WARN("fail to init sstable redo writer", K(ret));
|
||||||
reshaped_row.flag_.set_flag(ObDmlFlag::DF_INSERT);
|
} else if (OB_UNLIKELY(nullptr == static_cast<ObComplementDataDag *>(get_dag()))) {
|
||||||
const ObColDescIArray &col_descs = data_desc.col_desc_array_;
|
ret = OB_ERR_UNEXPECTED;
|
||||||
ObArenaAllocator allocator(lib::ObLabel("CompDataTaskTmp"));
|
LOG_WARN("the dag of this task is null", K(ret));
|
||||||
ObTableSchemaParam schema_param(allocator);
|
} else if (FALSE_IT(sstable_redo_writer.set_start_scn(
|
||||||
ObRelativeTable relative_table;
|
static_cast<ObComplementDataDag *>(get_dag())->get_context().data_sstable_redo_writer_.get_start_scn()))) {
|
||||||
ObRowReshape *reshape_ptr = nullptr;
|
} else if (OB_FAIL(callback.init(DDL_MB_DATA_TYPE, hidden_table_key, &sstable_redo_writer))) {
|
||||||
ObSQLMode sql_mode_for_ddl_reshape = SMO_TRADITIONAL;
|
LOG_WARN("fail to init data callback", K(ret), K(hidden_table_key));
|
||||||
ObDatumRow datum_row;
|
} else if (OB_FAIL(writer.open(data_desc, macro_start_seq, &callback))) {
|
||||||
// Hack to prevent row reshaping from converting empty string to null.
|
LOG_WARN("fail to open macro block writer", K(ret), K(data_desc));
|
||||||
//
|
} else {
|
||||||
// Supposing we have a row of type varchar with some spaces and an index on this column,
|
rowkey_column_cnt = hidden_table_schema->get_rowkey_column_num();
|
||||||
// and then we convert this column to char. In this case, the DDL routine will first rebuild
|
}
|
||||||
// the data table and then rebuilding the index table. The row may be reshaped as follows.
|
|
||||||
//
|
ObRelativeTable relative_table;
|
||||||
// - without hack: ' '(varchar) => ''(char) => null(char)
|
ObTableSchemaParam schema_param(allocator);
|
||||||
// - with hack: ' '(varchar) => ''(char) => ''(char)
|
// Hack to prevent row reshaping from converting empty string to null.
|
||||||
if (OB_FAIL(ret)) {
|
//
|
||||||
} else if (OB_FAIL(schema_param.convert(param_->hidden_table_schema_))) {
|
// Supposing we have a row of type varchar with some spaces and an index on this column,
|
||||||
LOG_WARN("failed to convert schema param", K(ret));
|
// and then we convert this column to char. In this case, the DDL routine will first rebuild
|
||||||
if (OB_SCHEMA_ERROR == ret) {
|
// the data table and then rebuilding the index table. The row may be reshaped as follows.
|
||||||
ret = OB_CANCELED;
|
//
|
||||||
|
// - without hack: ' '(varchar) => ''(char) => null(char)
|
||||||
|
// - with hack: ' '(varchar) => ''(char) => ''(char)
|
||||||
|
if (OB_FAIL(ret)) {
|
||||||
|
} else if (OB_FAIL(schema_param.convert(hidden_table_schema))) {
|
||||||
|
LOG_WARN("failed to convert schema param", K(ret));
|
||||||
|
if (OB_SCHEMA_ERROR == ret) {
|
||||||
|
ret = OB_CANCELED;
|
||||||
|
}
|
||||||
|
} else if (OB_FAIL(relative_table.init(&schema_param, param_->dest_tablet_id_))) {
|
||||||
|
LOG_WARN("fail to init relative_table", K(ret), K(schema_param), K(param_->dest_tablet_id_));
|
||||||
|
} else if (OB_FAIL(ObRowReshapeUtil::malloc_rows_reshape_if_need(
|
||||||
|
allocator, data_desc.col_desc_array_, 1, relative_table, sql_mode_for_ddl_reshape, reshape_ptr))) {
|
||||||
|
LOG_WARN("failed to malloc row reshape", K(ret));
|
||||||
|
} else if (OB_FAIL(datum_row.init(allocator, data_desc.col_desc_array_.count()))) {
|
||||||
|
LOG_WARN("failed to init datum row", K(ret), K(data_desc.col_desc_array_));
|
||||||
}
|
}
|
||||||
} else if (OB_FAIL(relative_table.init(&schema_param, param_->dest_tablet_id_))) {
|
|
||||||
LOG_WARN("fail to init relative_table", K(ret), K(schema_param), K(param_->dest_tablet_id_));
|
|
||||||
} else if (OB_FAIL(ObRowReshapeUtil::malloc_rows_reshape_if_need(
|
|
||||||
allocator, col_descs, 1, relative_table, sql_mode_for_ddl_reshape, reshape_ptr))) {
|
|
||||||
LOG_WARN("failed to malloc row reshape", K(ret));
|
|
||||||
} else if (OB_FAIL(datum_row.init(allocator, col_descs.count()))) {
|
|
||||||
LOG_WARN("failed to init datum row", K(ret), K(col_descs));
|
|
||||||
}
|
}
|
||||||
|
|
||||||
while (OB_SUCC(ret)) { //get each row from row_iter
|
while (OB_SUCC(ret)) { //get each row from row_iter
|
||||||
const ObDatumRow *tmp_row = nullptr;
|
const ObDatumRow *tmp_row = nullptr;
|
||||||
ObStoreRow tmp_store_row;
|
ObStoreRow tmp_store_row;
|
||||||
@ -1065,11 +1080,11 @@ int ObComplementWriteTask::append_row(ObLocalScan &local_scan)
|
|||||||
}
|
}
|
||||||
if (OB_FAIL(ret)) {
|
if (OB_FAIL(ret)) {
|
||||||
// do nothing
|
// do nothing
|
||||||
} else if (OB_FAIL(add_extra_rowkey(extra_rowkey_cnt, *tmp_row))) {
|
} else if (OB_FAIL(add_extra_rowkey(rowkey_column_cnt, extra_rowkey_cnt, *tmp_row))) {
|
||||||
LOG_WARN("fail to add extra rowkey", K(ret));
|
LOG_WARN("fail to add extra rowkey", K(ret));
|
||||||
} else if (OB_FAIL(write_row_.to_store_row(col_descs, tmp_store_row))) {
|
} else if (OB_FAIL(write_row_.to_store_row(data_desc.col_desc_array_, tmp_store_row))) {
|
||||||
} else if (OB_FAIL(ObRowReshapeUtil::reshape_table_rows(
|
} else if (OB_FAIL(ObRowReshapeUtil::reshape_table_rows(
|
||||||
&tmp_store_row.row_val_, reshape_ptr, col_descs.count(), &reshaped_row, 1, sql_mode_for_ddl_reshape))) {
|
&tmp_store_row.row_val_, reshape_ptr, data_desc.col_desc_array_.count(), &reshaped_row, 1, sql_mode_for_ddl_reshape))) {
|
||||||
LOG_WARN("failed to malloc and reshape row", K(ret));
|
LOG_WARN("failed to malloc and reshape row", K(ret));
|
||||||
} else if (OB_FAIL(datum_row.from_store_row(reshaped_row))) {
|
} else if (OB_FAIL(datum_row.from_store_row(reshaped_row))) {
|
||||||
STORAGE_LOG(WARN, "Failed to transfer store row ", K(ret), K(reshaped_row));
|
STORAGE_LOG(WARN, "Failed to transfer store row ", K(ret), K(reshaped_row));
|
||||||
@ -1105,8 +1120,8 @@ int ObComplementWriteTask::append_row(ObLocalScan &local_scan)
|
|||||||
LOG_WARN("fail to close writer", K(ret));
|
LOG_WARN("fail to close writer", K(ret));
|
||||||
} else if (OB_FAIL(local_scan.get_origin_table_checksum(report_col_checksums, report_col_ids))) {
|
} else if (OB_FAIL(local_scan.get_origin_table_checksum(report_col_checksums, report_col_ids))) {
|
||||||
LOG_WARN("fail to get origin table columns checksum", K(ret));
|
LOG_WARN("fail to get origin table columns checksum", K(ret));
|
||||||
} else if (OB_FAIL(ObDDLChecksumOperator::update_checksum(param_->data_table_schema_->get_tenant_id(),
|
} else if (OB_FAIL(ObDDLChecksumOperator::update_checksum(param_->tenant_id_,
|
||||||
param_->data_table_schema_->get_table_id() /* data_table_id */,
|
param_->source_table_id_,
|
||||||
param_->task_id_,
|
param_->task_id_,
|
||||||
report_col_checksums,
|
report_col_checksums,
|
||||||
report_col_ids,
|
report_col_ids,
|
||||||
@ -1160,8 +1175,8 @@ int ObComplementMergeTask::process()
|
|||||||
} else if (FALSE_IT(dag = static_cast<ObComplementDataDag *>(tmp_dag))) {
|
} else if (FALSE_IT(dag = static_cast<ObComplementDataDag *>(tmp_dag))) {
|
||||||
} else if (OB_SUCCESS != (context_->complement_data_ret_)) {
|
} else if (OB_SUCCESS != (context_->complement_data_ret_)) {
|
||||||
LOG_WARN("complement data has already failed", "ret", context_->complement_data_ret_);
|
LOG_WARN("complement data has already failed", "ret", context_->complement_data_ret_);
|
||||||
} else if (OB_FAIL(guard.switch_to(param_->hidden_table_schema_->get_tenant_id()))) {
|
} else if (OB_FAIL(guard.switch_to(param_->tenant_id_))) {
|
||||||
LOG_WARN("switch to tenant failed", K(ret), K(param_->hidden_table_schema_->get_tenant_id()));
|
LOG_WARN("switch to tenant failed", K(ret), K(param_->tenant_id_));
|
||||||
} else if (context_->is_major_sstable_exist_) {
|
} else if (context_->is_major_sstable_exist_) {
|
||||||
const ObSSTable *latest_major_sstable = nullptr;
|
const ObSSTable *latest_major_sstable = nullptr;
|
||||||
if (OB_FAIL(ObTabletDDLUtil::check_and_get_major_sstable(param_->ls_id_, param_->dest_tablet_id_, latest_major_sstable))) {
|
if (OB_FAIL(ObTabletDDLUtil::check_and_get_major_sstable(param_->ls_id_, param_->dest_tablet_id_, latest_major_sstable))) {
|
||||||
@ -1171,7 +1186,7 @@ int ObComplementMergeTask::process()
|
|||||||
LOG_WARN("unexpected error, major sstable shoud not be null", K(ret), K(*param_));
|
LOG_WARN("unexpected error, major sstable shoud not be null", K(ret), K(*param_));
|
||||||
} else if (OB_FAIL(ObTabletDDLUtil::report_ddl_checksum(param_->ls_id_,
|
} else if (OB_FAIL(ObTabletDDLUtil::report_ddl_checksum(param_->ls_id_,
|
||||||
param_->dest_tablet_id_,
|
param_->dest_tablet_id_,
|
||||||
param_->hidden_table_schema_->get_table_id(),
|
param_->dest_table_id_,
|
||||||
1 /* execution_id */,
|
1 /* execution_id */,
|
||||||
param_->task_id_,
|
param_->task_id_,
|
||||||
latest_major_sstable->get_meta().get_col_checksum()))) {
|
latest_major_sstable->get_meta().get_col_checksum()))) {
|
||||||
@ -1227,7 +1242,7 @@ int ObComplementMergeTask::add_build_hidden_table_sstable()
|
|||||||
|
|
||||||
if (OB_FAIL(ret)) {
|
if (OB_FAIL(ret)) {
|
||||||
} else if (OB_FAIL(context_->data_sstable_redo_writer_.write_commit_log(hidden_table_key,
|
} else if (OB_FAIL(context_->data_sstable_redo_writer_.write_commit_log(hidden_table_key,
|
||||||
param_->hidden_table_schema_->get_table_id(),
|
param_->dest_table_id_,
|
||||||
param_->execution_id_,
|
param_->execution_id_,
|
||||||
param_->task_id_,
|
param_->task_id_,
|
||||||
commit_scn))) {
|
commit_scn))) {
|
||||||
@ -1242,7 +1257,7 @@ int ObComplementMergeTask::add_build_hidden_table_sstable()
|
|||||||
const ObLSID &ls_id = param_->ls_id_;
|
const ObLSID &ls_id = param_->ls_id_;
|
||||||
const ObTabletID &tablet_id = tablet->get_tablet_meta().tablet_id_;
|
const ObTabletID &tablet_id = tablet->get_tablet_meta().tablet_id_;
|
||||||
const SCN &ddl_start_scn = static_cast<ObComplementDataDag *>(get_dag())->get_context().data_sstable_redo_writer_.get_start_scn();
|
const SCN &ddl_start_scn = static_cast<ObComplementDataDag *>(get_dag())->get_context().data_sstable_redo_writer_.get_start_scn();
|
||||||
const uint64_t table_id = param_->hidden_table_schema_->get_table_id();
|
const uint64_t table_id = param_->dest_table_id_;
|
||||||
const int64_t ddl_task_id = param_->task_id_;
|
const int64_t ddl_task_id = param_->task_id_;
|
||||||
if (OB_FAIL(tablet->get_ddl_kv_mgr(ddl_kv_mgr_handle))) {
|
if (OB_FAIL(tablet->get_ddl_kv_mgr(ddl_kv_mgr_handle))) {
|
||||||
LOG_WARN("get ddl kv manager failed", K(ret));
|
LOG_WARN("get ddl kv manager failed", K(ret));
|
||||||
@ -1264,9 +1279,9 @@ int ObComplementMergeTask::add_build_hidden_table_sstable()
|
|||||||
* -----------------------------------ObLocalScan-----------------------------------------
|
* -----------------------------------ObLocalScan-----------------------------------------
|
||||||
*/
|
*/
|
||||||
|
|
||||||
ObLocalScan::ObLocalScan() : is_inited_(false), extended_gc_(), data_table_schema_(nullptr),
|
ObLocalScan::ObLocalScan() : is_inited_(false), tenant_id_(OB_INVALID_TENANT_ID), source_table_id_(OB_INVALID_ID),
|
||||||
hidden_table_schema_(nullptr), snapshot_version_(common::OB_INVALID_VERSION), txs_(nullptr),
|
dest_table_id_(OB_INVALID_ID), schema_version_(0), extended_gc_(), snapshot_version_(common::OB_INVALID_VERSION),
|
||||||
default_row_(), tmp_row_(), row_iter_(nullptr), scan_merge_(nullptr), ctx_(), access_param_(),
|
txs_(nullptr), default_row_(), tmp_row_(), row_iter_(nullptr), scan_merge_(nullptr), ctx_(), access_param_(),
|
||||||
access_ctx_(), get_table_param_(), allocator_("ObLocalScan"), calc_buf_(ObModIds::OB_SQL_EXPR_CALC),
|
access_ctx_(), get_table_param_(), allocator_("ObLocalScan"), calc_buf_(ObModIds::OB_SQL_EXPR_CALC),
|
||||||
col_params_(), read_info_(), exist_column_mapping_(allocator_), checksum_calculator_()
|
col_params_(), read_info_(), exist_column_mapping_(allocator_), checksum_calculator_()
|
||||||
{}
|
{}
|
||||||
@ -1294,10 +1309,10 @@ int ObLocalScan::init(
|
|||||||
const ObIArray<share::schema::ObColDesc> &col_ids,
|
const ObIArray<share::schema::ObColDesc> &col_ids,
|
||||||
const ObIArray<share::schema::ObColDesc> &org_col_ids,
|
const ObIArray<share::schema::ObColDesc> &org_col_ids,
|
||||||
const ObIArray<int32_t> &projector,
|
const ObIArray<int32_t> &projector,
|
||||||
const ObTableSchema *data_table_schema,
|
const ObTableSchema &data_table_schema,
|
||||||
const int64_t snapshot_version,
|
const int64_t snapshot_version,
|
||||||
ObTransService *txs,
|
ObTransService *txs,
|
||||||
const ObTableSchema *hidden_table_schema,
|
const ObTableSchema &hidden_table_schema,
|
||||||
const bool output_org_cols_only)
|
const bool output_org_cols_only)
|
||||||
{
|
{
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
@ -1305,17 +1320,15 @@ int ObLocalScan::init(
|
|||||||
ret = OB_INIT_TWICE;
|
ret = OB_INIT_TWICE;
|
||||||
LOG_WARN("ObLocalScan has been initialized before", K(ret));
|
LOG_WARN("ObLocalScan has been initialized before", K(ret));
|
||||||
} else if (org_col_ids.count() < 1 || col_ids.count() < 1 || projector.count() < 1
|
} else if (org_col_ids.count() < 1 || col_ids.count() < 1 || projector.count() < 1
|
||||||
|| OB_ISNULL(data_table_schema) || OB_ISNULL(hidden_table_schema) || snapshot_version < 1 || OB_ISNULL(txs)) {
|
|| !data_table_schema.is_valid() || !hidden_table_schema.is_valid() || snapshot_version < 1 || OB_ISNULL(txs)) {
|
||||||
ret = OB_INVALID_ARGUMENT;
|
ret = OB_INVALID_ARGUMENT;
|
||||||
LOG_WARN("invalid auguments", K(ret), KP(data_table_schema), KP(hidden_table_schema),
|
LOG_WARN("invalid auguments", K(ret), K(data_table_schema), K(hidden_table_schema),
|
||||||
K(col_ids), K(org_col_ids), K(projector), K(snapshot_version));
|
K(col_ids), K(org_col_ids), K(projector), K(snapshot_version));
|
||||||
} else {
|
} else {
|
||||||
data_table_schema_ = data_table_schema;
|
|
||||||
hidden_table_schema_ = hidden_table_schema;
|
|
||||||
snapshot_version_ = snapshot_version;
|
snapshot_version_ = snapshot_version;
|
||||||
txs_ = txs;
|
txs_ = txs;
|
||||||
output_org_cols_only_ = output_org_cols_only;
|
output_org_cols_only_ = output_org_cols_only;
|
||||||
if (OB_FAIL(check_generated_column_exist(org_col_ids))) {
|
if (OB_FAIL(check_generated_column_exist(hidden_table_schema, org_col_ids))) {
|
||||||
LOG_WARN("fail to init generated columns", K(ret), K(org_col_ids));
|
LOG_WARN("fail to init generated columns", K(ret), K(org_col_ids));
|
||||||
} else if (OB_FAIL(extended_gc_.extended_col_ids_.assign(col_ids))) {
|
} else if (OB_FAIL(extended_gc_.extended_col_ids_.assign(col_ids))) {
|
||||||
LOG_WARN("fail to assign", K(ret));
|
LOG_WARN("fail to assign", K(ret));
|
||||||
@ -1327,7 +1340,7 @@ int ObLocalScan::init(
|
|||||||
STORAGE_LOG(WARN, "Failed to init datum row", K(ret));
|
STORAGE_LOG(WARN, "Failed to init datum row", K(ret));
|
||||||
} else if (OB_FAIL(tmp_row_.init(allocator_, org_col_ids.count()))) {
|
} else if (OB_FAIL(tmp_row_.init(allocator_, org_col_ids.count()))) {
|
||||||
STORAGE_LOG(WARN, "Failed to init datum row", K(ret));
|
STORAGE_LOG(WARN, "Failed to init datum row", K(ret));
|
||||||
} else if (OB_FAIL(get_exist_column_mapping())){
|
} else if (OB_FAIL(get_exist_column_mapping(data_table_schema, hidden_table_schema))){
|
||||||
LOG_WARN("fail to init positions for resolving row", K(ret));
|
LOG_WARN("fail to init positions for resolving row", K(ret));
|
||||||
} else if (OB_FAIL(checksum_calculator_.init(extended_gc_.org_extended_col_ids_.count()
|
} else if (OB_FAIL(checksum_calculator_.init(extended_gc_.org_extended_col_ids_.count()
|
||||||
+ storage::ObMultiVersionRowkeyHelpper::get_extra_rowkey_col_cnt()))) {
|
+ storage::ObMultiVersionRowkeyHelpper::get_extra_rowkey_col_cnt()))) {
|
||||||
@ -1335,10 +1348,13 @@ int ObLocalScan::init(
|
|||||||
} else {
|
} else {
|
||||||
default_row_.row_flag_.set_flag(ObDmlFlag::DF_INSERT);
|
default_row_.row_flag_.set_flag(ObDmlFlag::DF_INSERT);
|
||||||
tmp_row_.row_flag_.set_flag(ObDmlFlag::DF_INSERT);
|
tmp_row_.row_flag_.set_flag(ObDmlFlag::DF_INSERT);
|
||||||
uint64_t tenant_id = hidden_table_schema->get_tenant_id();
|
if (OB_FAIL(hidden_table_schema.get_orig_default_row(org_col_ids, default_row_))) {
|
||||||
if (OB_FAIL(hidden_table_schema->get_orig_default_row(org_col_ids, default_row_))) {
|
|
||||||
LOG_WARN("fail to get default row from table schema", K(ret));
|
LOG_WARN("fail to get default row from table schema", K(ret));
|
||||||
} else {
|
} else {
|
||||||
|
tenant_id_ = data_table_schema.get_tenant_id();
|
||||||
|
source_table_id_ = data_table_schema.get_table_id();
|
||||||
|
dest_table_id_ = hidden_table_schema.get_table_id();
|
||||||
|
schema_version_ = hidden_table_schema.get_schema_version();
|
||||||
is_inited_ = true;
|
is_inited_ = true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -1346,7 +1362,9 @@ int ObLocalScan::init(
|
|||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
int ObLocalScan::get_output_columns(ObIArray<ObColDesc> &col_ids)
|
int ObLocalScan::get_output_columns(
|
||||||
|
const ObTableSchema &hidden_table_schema,
|
||||||
|
ObIArray<ObColDesc> &col_ids)
|
||||||
{
|
{
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
if (output_org_cols_only_) {
|
if (output_org_cols_only_) {
|
||||||
@ -1354,21 +1372,24 @@ int ObLocalScan::get_output_columns(ObIArray<ObColDesc> &col_ids)
|
|||||||
LOG_WARN("assign tmp col ids failed", K(ret));
|
LOG_WARN("assign tmp col ids failed", K(ret));
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
if (OB_FAIL(hidden_table_schema_->get_store_column_ids(col_ids, false))) {
|
if (OB_FAIL(hidden_table_schema.get_store_column_ids(col_ids, false))) {
|
||||||
LOG_WARN("fail to get column ids", K(ret), K(*hidden_table_schema_));
|
LOG_WARN("fail to get column ids", K(ret), K(hidden_table_schema));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
// record the position of data table columns in hidden table by exist_column_mapping_.
|
// record the position of data table columns in hidden table by exist_column_mapping_.
|
||||||
int ObLocalScan::get_exist_column_mapping()
|
int ObLocalScan::get_exist_column_mapping(
|
||||||
|
const ObTableSchema &data_table_schema,
|
||||||
|
const ObTableSchema &hidden_table_schema)
|
||||||
{
|
{
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
void *buf = nullptr;
|
void *buf = nullptr;
|
||||||
ObArray<ObColDesc> tmp_col_ids;
|
ObArray<ObColDesc> tmp_col_ids;
|
||||||
|
|
||||||
if (OB_FAIL(get_output_columns(tmp_col_ids))) {
|
if (OB_FAIL(get_output_columns(hidden_table_schema, tmp_col_ids))) {
|
||||||
|
LOG_WARN("get output columns failed", K(ret), K(hidden_table_schema));
|
||||||
} else if (exist_column_mapping_.is_inited() && OB_FAIL(exist_column_mapping_.expand_size(tmp_col_ids.count()))) {
|
} else if (exist_column_mapping_.is_inited() && OB_FAIL(exist_column_mapping_.expand_size(tmp_col_ids.count()))) {
|
||||||
LOG_WARN("fail to expand size of bitmap", K(ret));
|
LOG_WARN("fail to expand size of bitmap", K(ret));
|
||||||
} else if (!exist_column_mapping_.is_inited() && OB_FAIL(exist_column_mapping_.init(tmp_col_ids.count(), false))) {
|
} else if (!exist_column_mapping_.is_inited() && OB_FAIL(exist_column_mapping_.init(tmp_col_ids.count(), false))) {
|
||||||
@ -1376,9 +1397,9 @@ int ObLocalScan::get_exist_column_mapping()
|
|||||||
} else {
|
} else {
|
||||||
exist_column_mapping_.reuse(false);
|
exist_column_mapping_.reuse(false);
|
||||||
for (int64_t i = 0; OB_SUCC(ret) && i < tmp_col_ids.count(); i++) {
|
for (int64_t i = 0; OB_SUCC(ret) && i < tmp_col_ids.count(); i++) {
|
||||||
const ObColumnSchemaV2 *hidden_column_schema = hidden_table_schema_->get_column_schema(tmp_col_ids.at(i).col_id_);
|
const ObColumnSchemaV2 *hidden_column_schema = hidden_table_schema.get_column_schema(tmp_col_ids.at(i).col_id_);
|
||||||
const ObString &hidden_column_name = hidden_column_schema->get_column_name_str();
|
const ObString &hidden_column_name = hidden_column_schema->get_column_name_str();
|
||||||
const ObColumnSchemaV2 *data_column_schema = data_table_schema_->get_column_schema(hidden_column_name);
|
const ObColumnSchemaV2 *data_column_schema = data_table_schema.get_column_schema(hidden_column_name);
|
||||||
if (nullptr == data_column_schema) {
|
if (nullptr == data_column_schema) {
|
||||||
// newly added column, can not find in data table.
|
// newly added column, can not find in data table.
|
||||||
} else if (OB_FAIL(exist_column_mapping_.set(i))) {
|
} else if (OB_FAIL(exist_column_mapping_.set(i))) {
|
||||||
@ -1390,12 +1411,13 @@ int ObLocalScan::get_exist_column_mapping()
|
|||||||
}
|
}
|
||||||
|
|
||||||
int ObLocalScan::check_generated_column_exist(
|
int ObLocalScan::check_generated_column_exist(
|
||||||
|
const ObTableSchema &hidden_table_schema,
|
||||||
const ObIArray<share::schema::ObColDesc> &org_col_ids)
|
const ObIArray<share::schema::ObColDesc> &org_col_ids)
|
||||||
{
|
{
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
for (int64_t i = 0; OB_SUCC(ret) && i < org_col_ids.count(); ++i) {
|
for (int64_t i = 0; OB_SUCC(ret) && i < org_col_ids.count(); ++i) {
|
||||||
const ObColumnSchemaV2 *column_schema = nullptr;
|
const ObColumnSchemaV2 *column_schema = nullptr;
|
||||||
if (OB_ISNULL(column_schema = hidden_table_schema_->get_column_schema(org_col_ids.at(i).col_id_))) {
|
if (OB_ISNULL(column_schema = hidden_table_schema.get_column_schema(org_col_ids.at(i).col_id_))) {
|
||||||
ret = OB_ERR_UNEXPECTED;
|
ret = OB_ERR_UNEXPECTED;
|
||||||
LOG_WARN("The column schema should not be null", K(ret), K(org_col_ids.at(i)));
|
LOG_WARN("The column schema should not be null", K(ret), K(org_col_ids.at(i)));
|
||||||
} else if (OB_UNLIKELY(column_schema->is_stored_generated_column())) {
|
} else if (OB_UNLIKELY(column_schema->is_stored_generated_column())) {
|
||||||
@ -1407,6 +1429,7 @@ int ObLocalScan::check_generated_column_exist(
|
|||||||
}
|
}
|
||||||
|
|
||||||
int ObLocalScan::table_scan(
|
int ObLocalScan::table_scan(
|
||||||
|
const ObTableSchema &data_table_schema,
|
||||||
const share::ObLSID &ls_id,
|
const share::ObLSID &ls_id,
|
||||||
const ObTabletID &tablet_id,
|
const ObTabletID &tablet_id,
|
||||||
ObTabletTableIterator &table_iter,
|
ObTabletTableIterator &table_iter,
|
||||||
@ -1416,9 +1439,9 @@ int ObLocalScan::table_scan(
|
|||||||
{
|
{
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
const ObTableReadInfo &full_read_info = table_iter.tablet_handle_.get_obj()->get_full_read_info();
|
const ObTableReadInfo &full_read_info = table_iter.tablet_handle_.get_obj()->get_full_read_info();
|
||||||
if (OB_FAIL(construct_column_schema())) {
|
if (OB_FAIL(construct_column_schema(data_table_schema))) {
|
||||||
LOG_WARN("fail to construct column schema", K(ret), K(col_params_));
|
LOG_WARN("fail to construct column schema", K(ret), K(col_params_));
|
||||||
} else if (OB_FAIL(construct_access_param(tablet_id, full_read_info))) {
|
} else if (OB_FAIL(construct_access_param(data_table_schema, tablet_id, full_read_info))) {
|
||||||
LOG_WARN("fail to construct access param", K(ret), K(col_params_));
|
LOG_WARN("fail to construct access param", K(ret), K(col_params_));
|
||||||
} else if (OB_FAIL(construct_range_ctx(query_flag, ls_id, tx_desc))) {
|
} else if (OB_FAIL(construct_range_ctx(query_flag, ls_id, tx_desc))) {
|
||||||
LOG_WARN("fail to construct range ctx", K(ret), K(query_flag));
|
LOG_WARN("fail to construct range ctx", K(ret), K(query_flag));
|
||||||
@ -1431,12 +1454,12 @@ int ObLocalScan::table_scan(
|
|||||||
}
|
}
|
||||||
|
|
||||||
//convert column schema to column param
|
//convert column schema to column param
|
||||||
int ObLocalScan::construct_column_schema()
|
int ObLocalScan::construct_column_schema(const ObTableSchema &data_table_schema)
|
||||||
{
|
{
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
ObArray<ObColDesc> &extended_col_ids = extended_gc_.extended_col_ids_;
|
ObArray<ObColDesc> &extended_col_ids = extended_gc_.extended_col_ids_;
|
||||||
for (int64_t i = 0; OB_SUCC(ret) && i < extended_col_ids.count(); i++) {
|
for (int64_t i = 0; OB_SUCC(ret) && i < extended_col_ids.count(); i++) {
|
||||||
const ObColumnSchemaV2 *col = data_table_schema_->get_column_schema(extended_col_ids.at(i).col_id_);
|
const ObColumnSchemaV2 *col = data_table_schema.get_column_schema(extended_col_ids.at(i).col_id_);
|
||||||
if (OB_ISNULL(col)) {
|
if (OB_ISNULL(col)) {
|
||||||
ret = OB_ERR_UNEXPECTED;
|
ret = OB_ERR_UNEXPECTED;
|
||||||
LOG_WARN("fail to get column schema", K(ret), K(extended_col_ids.at(i).col_id_));
|
LOG_WARN("fail to get column schema", K(ret), K(extended_col_ids.at(i).col_id_));
|
||||||
@ -1476,6 +1499,7 @@ int ObLocalScan::construct_column_schema()
|
|||||||
|
|
||||||
//construct table access param
|
//construct table access param
|
||||||
int ObLocalScan::construct_access_param(
|
int ObLocalScan::construct_access_param(
|
||||||
|
const ObTableSchema &data_table_schema,
|
||||||
const ObTabletID &tablet_id,
|
const ObTabletID &tablet_id,
|
||||||
const ObTableReadInfo &full_read_info)
|
const ObTableReadInfo &full_read_info)
|
||||||
{
|
{
|
||||||
@ -1485,7 +1509,7 @@ int ObLocalScan::construct_access_param(
|
|||||||
ObArray<ObColDesc> tmp_col_ids;
|
ObArray<ObColDesc> tmp_col_ids;
|
||||||
bool is_oracle_mode = false;
|
bool is_oracle_mode = false;
|
||||||
// to construct column index, i.e., cols_index.
|
// to construct column index, i.e., cols_index.
|
||||||
if (OB_FAIL(data_table_schema_->get_store_column_ids(tmp_col_ids, false))) {
|
if (OB_FAIL(data_table_schema.get_store_column_ids(tmp_col_ids, false))) {
|
||||||
LOG_WARN("fail to get store columns id", K(ret), K(tmp_col_ids));
|
LOG_WARN("fail to get store columns id", K(ret), K(tmp_col_ids));
|
||||||
} else {
|
} else {
|
||||||
for (int64_t i = 0; OB_SUCC(ret) && i < extended_gc_.extended_col_ids_.count(); i++) {
|
for (int64_t i = 0; OB_SUCC(ret) && i < extended_gc_.extended_col_ids_.count(); i++) {
|
||||||
@ -1502,7 +1526,7 @@ int ObLocalScan::construct_access_param(
|
|||||||
if (OB_SUCC(ret) && !is_found) {
|
if (OB_SUCC(ret) && !is_found) {
|
||||||
ret = OB_ERR_UNEXPECTED;
|
ret = OB_ERR_UNEXPECTED;
|
||||||
LOG_WARN("error unexpected, column is not in data table", K(ret),
|
LOG_WARN("error unexpected, column is not in data table", K(ret),
|
||||||
K(extended_gc_.extended_col_ids_.at(i)), K(tmp_col_ids), KPC(data_table_schema_));
|
K(extended_gc_.extended_col_ids_.at(i)), K(tmp_col_ids), K(data_table_schema));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -1510,11 +1534,11 @@ int ObLocalScan::construct_access_param(
|
|||||||
} else if (cols_index.count() != extended_gc_.extended_col_ids_.count()) {
|
} else if (cols_index.count() != extended_gc_.extended_col_ids_.count()) {
|
||||||
ret = OB_ERR_UNEXPECTED;
|
ret = OB_ERR_UNEXPECTED;
|
||||||
LOG_WARN("unexpected error", K(ret), K(cols_index), K(extended_gc_));
|
LOG_WARN("unexpected error", K(ret), K(cols_index), K(extended_gc_));
|
||||||
} else if (OB_FAIL(data_table_schema_->check_if_oracle_compat_mode(is_oracle_mode))) {
|
} else if (OB_FAIL(data_table_schema.check_if_oracle_compat_mode(is_oracle_mode))) {
|
||||||
STORAGE_LOG(WARN, "Failed to check oralce mode", K(ret));
|
STORAGE_LOG(WARN, "Failed to check oralce mode", K(ret));
|
||||||
} else if (OB_FAIL(read_info_.init(allocator_,
|
} else if (OB_FAIL(read_info_.init(allocator_,
|
||||||
data_table_schema_->get_column_count(),
|
data_table_schema.get_column_count(),
|
||||||
data_table_schema_->get_rowkey_column_num(),
|
data_table_schema.get_rowkey_column_num(),
|
||||||
is_oracle_mode,
|
is_oracle_mode,
|
||||||
extended_gc_.extended_col_ids_, // TODO @yiren, remove column id.
|
extended_gc_.extended_col_ids_, // TODO @yiren, remove column id.
|
||||||
false /*is_multi_version_full*/,
|
false /*is_multi_version_full*/,
|
||||||
@ -1525,7 +1549,7 @@ int ObLocalScan::construct_access_param(
|
|||||||
ObArray<ObColDesc> &extended_col_ids = extended_gc_.extended_col_ids_;
|
ObArray<ObColDesc> &extended_col_ids = extended_gc_.extended_col_ids_;
|
||||||
ObArray<int32_t> &output_projector = extended_gc_.output_projector_;
|
ObArray<int32_t> &output_projector = extended_gc_.output_projector_;
|
||||||
access_param_.iter_param_.tablet_id_ = tablet_id;
|
access_param_.iter_param_.tablet_id_ = tablet_id;
|
||||||
access_param_.iter_param_.table_id_ = data_table_schema_->get_table_id();
|
access_param_.iter_param_.table_id_ = data_table_schema.get_table_id();
|
||||||
access_param_.iter_param_.out_cols_project_ = &output_projector;
|
access_param_.iter_param_.out_cols_project_ = &output_projector;
|
||||||
access_param_.iter_param_.read_info_ = &read_info_;
|
access_param_.iter_param_.read_info_ = &read_info_;
|
||||||
access_param_.iter_param_.full_read_info_ = &full_read_info;
|
access_param_.iter_param_.full_read_info_ = &full_read_info;
|
||||||
@ -1605,29 +1629,49 @@ int ObLocalScan::construct_multiple_scan_merge(
|
|||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
int ObLocalScan::get_origin_table_checksum(ObArray<int64_t> &report_col_checksums, ObArray<int64_t> &report_col_ids)
|
int ObLocalScan::get_origin_table_checksum(
|
||||||
|
ObArray<int64_t> &report_col_checksums,
|
||||||
|
ObArray<int64_t> &report_col_ids)
|
||||||
{
|
{
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
report_col_checksums.reuse();
|
report_col_checksums.reuse();
|
||||||
report_col_ids.reuse();
|
report_col_ids.reuse();
|
||||||
ObArray<ObColDesc> tmp_col_ids;
|
ObArray<ObColDesc> tmp_col_ids;
|
||||||
const int64_t rowkey_cols_cnt = hidden_table_schema_->get_rowkey_column_num();
|
ObSchemaGetterGuard schema_guard;
|
||||||
const int64_t extra_rowkey_cnt = storage::ObMultiVersionRowkeyHelpper::get_extra_rowkey_col_cnt();
|
const ObTableSchema *data_table_schema = nullptr;
|
||||||
|
const ObTableSchema *hidden_table_schema = nullptr;
|
||||||
if (OB_UNLIKELY(!is_inited_)) {
|
if (OB_UNLIKELY(!is_inited_)) {
|
||||||
ret = OB_NOT_INIT;
|
ret = OB_NOT_INIT;
|
||||||
LOG_WARN("not init", K(ret));
|
LOG_WARN("not init", K(ret));
|
||||||
} else if (OB_FAIL(get_output_columns(tmp_col_ids))) {
|
} else if (OB_FAIL(ObMultiVersionSchemaService::get_instance().get_tenant_schema_guard(
|
||||||
|
tenant_id_, schema_guard, schema_version_))) {
|
||||||
|
LOG_WARN("fail to get tenant schema guard", K(ret), K(tenant_id_), K(schema_version_));
|
||||||
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_,
|
||||||
|
source_table_id_, data_table_schema))) {
|
||||||
|
LOG_WARN("get data table schema failed", K(ret), K(tenant_id_), K(source_table_id_));
|
||||||
|
} else if (OB_ISNULL(data_table_schema)) {
|
||||||
|
ret = OB_TABLE_NOT_EXIST;
|
||||||
|
LOG_WARN("data table not exist", K(ret), K(tenant_id_), K(source_table_id_));
|
||||||
|
} else if (OB_FAIL(schema_guard.get_table_schema(tenant_id_,
|
||||||
|
dest_table_id_, hidden_table_schema))) {
|
||||||
|
LOG_WARN("fail to get hidden table schema", K(ret), K(tenant_id_), K(dest_table_id_));
|
||||||
|
} else if (OB_ISNULL(hidden_table_schema)) {
|
||||||
|
ret = OB_TABLE_NOT_EXIST;
|
||||||
|
LOG_WARN("hidden table schema not exist", K(ret), K(tenant_id_), K(dest_table_id_));
|
||||||
|
} else if (OB_FAIL(get_output_columns(*hidden_table_schema, tmp_col_ids))) {
|
||||||
LOG_WARN("get output column failed", K(ret));
|
LOG_WARN("get output column failed", K(ret));
|
||||||
} else if (tmp_col_ids.size() != exist_column_mapping_.size()) {
|
} else if (tmp_col_ids.size() != exist_column_mapping_.size()) {
|
||||||
ret = OB_ERR_UNEXPECTED;
|
ret = OB_ERR_UNEXPECTED;
|
||||||
LOG_WARN("unexpected error", K(ret), K(tmp_col_ids), K(exist_column_mapping_.size()));
|
LOG_WARN("unexpected error", K(ret), K(tmp_col_ids), K(exist_column_mapping_.size()));
|
||||||
} else {
|
} else {
|
||||||
|
const int64_t rowkey_cols_cnt = hidden_table_schema->get_rowkey_column_num();
|
||||||
|
const int64_t extra_rowkey_cnt = storage::ObMultiVersionRowkeyHelpper::get_extra_rowkey_col_cnt();
|
||||||
// get data table columns id and corresponding checksum.
|
// get data table columns id and corresponding checksum.
|
||||||
for (int64_t i = 0; OB_SUCC(ret) && i < exist_column_mapping_.size(); i++) {
|
for (int64_t i = 0; OB_SUCC(ret) && i < exist_column_mapping_.size(); i++) {
|
||||||
if (exist_column_mapping_.test(i)) {
|
if (exist_column_mapping_.test(i)) {
|
||||||
const ObColumnSchemaV2 *hidden_col_schema = hidden_table_schema_->get_column_schema(tmp_col_ids.at(i).col_id_);
|
const ObColumnSchemaV2 *hidden_col_schema = hidden_table_schema->get_column_schema(tmp_col_ids.at(i).col_id_);
|
||||||
const ObString &hidden_column_name = hidden_col_schema->get_column_name_str();
|
const ObString &hidden_column_name = hidden_col_schema->get_column_name_str();
|
||||||
const ObColumnSchemaV2 *data_col_schema = data_table_schema_->get_column_schema(hidden_column_name);
|
const ObColumnSchemaV2 *data_col_schema = data_table_schema->get_column_schema(hidden_column_name);
|
||||||
const int64_t index_in_array = i < rowkey_cols_cnt ? i : i + extra_rowkey_cnt;
|
const int64_t index_in_array = i < rowkey_cols_cnt ? i : i + extra_rowkey_cnt;
|
||||||
if (OB_ISNULL(data_col_schema)) {
|
if (OB_ISNULL(data_col_schema)) {
|
||||||
ret = OB_ERR_UNEXPECTED;
|
ret = OB_ERR_UNEXPECTED;
|
||||||
|
|||||||
@ -38,21 +38,21 @@ struct ObComplementDataParam final
|
|||||||
public:
|
public:
|
||||||
static const int64_t DEFAULT_COMPLEMENT_DATA_MEMORY_LIMIT = 128L * 1024L * 1024L;
|
static const int64_t DEFAULT_COMPLEMENT_DATA_MEMORY_LIMIT = 128L * 1024L * 1024L;
|
||||||
ObComplementDataParam():
|
ObComplementDataParam():
|
||||||
is_inited_(false), tenant_id_(common::OB_INVALID_TENANT_ID), ls_id_(share::ObLSID::INVALID_LS_ID),
|
is_inited_(false), tenant_id_(common::OB_INVALID_TENANT_ID), ls_id_(share::ObLSID::INVALID_LS_ID),
|
||||||
source_tablet_id_(ObTabletID::INVALID_TABLET_ID), dest_tablet_id_(ObTabletID::INVALID_TABLET_ID),
|
source_table_id_(common::OB_INVALID_ID), dest_table_id_(common::OB_INVALID_ID),
|
||||||
data_table_schema_(nullptr), hidden_table_schema_(nullptr), allocator_("CompleteDataPar"),
|
source_tablet_id_(ObTabletID::INVALID_TABLET_ID), dest_tablet_id_(ObTabletID::INVALID_TABLET_ID), allocator_("CompleteDataPar"),
|
||||||
row_store_type_(common::ENCODING_ROW_STORE), schema_version_(0), snapshot_version_(0),
|
row_store_type_(common::ENCODING_ROW_STORE), schema_version_(0), snapshot_version_(0),
|
||||||
concurrent_cnt_(0), task_id_(0), execution_id_(-1), tablet_task_id_(0),
|
concurrent_cnt_(0), task_id_(0), execution_id_(-1), tablet_task_id_(0),
|
||||||
compat_mode_(lib::Worker::CompatMode::INVALID), cluster_version_(0)
|
compat_mode_(lib::Worker::CompatMode::INVALID), cluster_version_(0)
|
||||||
{}
|
{}
|
||||||
~ObComplementDataParam() { destroy(); }
|
~ObComplementDataParam() { destroy(); }
|
||||||
int init(const ObDDLBuildSingleReplicaRequestArg &arg);
|
int init(const ObDDLBuildSingleReplicaRequestArg &arg);
|
||||||
int split_task_ranges(const share::ObLSID &ls_id, const common::ObTabletID &tablet_id, const int64_t hint_parallelism);
|
int split_task_ranges(const share::ObLSID &ls_id, const common::ObTabletID &tablet_id, const int64_t tablet_size, const int64_t hint_parallelism);
|
||||||
int deep_copy_table_schemas(const share::schema::ObTableSchema *data_table_schema, const share::schema::ObTableSchema *hidden_table_schema);
|
|
||||||
bool is_valid() const
|
bool is_valid() const
|
||||||
{
|
{
|
||||||
return common::OB_INVALID_TENANT_ID != tenant_id_ && ls_id_.is_valid() && source_tablet_id_.is_valid()
|
return common::OB_INVALID_TENANT_ID != tenant_id_ && ls_id_.is_valid()
|
||||||
&& dest_tablet_id_.is_valid() && OB_NOT_NULL(data_table_schema_) && OB_NOT_NULL(hidden_table_schema_)
|
&& common::OB_INVALID_ID != source_table_id_ && common::OB_INVALID_ID != dest_table_id_
|
||||||
|
&& source_tablet_id_.is_valid() && dest_tablet_id_.is_valid()
|
||||||
&& 0 != concurrent_cnt_ && snapshot_version_ > 0 && compat_mode_ != lib::Worker::CompatMode::INVALID
|
&& 0 != concurrent_cnt_ && snapshot_version_ > 0 && compat_mode_ != lib::Worker::CompatMode::INVALID
|
||||||
&& execution_id_ >= 0 && tablet_task_id_ > 0 && cluster_version_ > 0;
|
&& execution_id_ >= 0 && tablet_task_id_ > 0 && cluster_version_ > 0;
|
||||||
}
|
}
|
||||||
@ -62,16 +62,10 @@ public:
|
|||||||
is_inited_ = false;
|
is_inited_ = false;
|
||||||
tenant_id_ = common::OB_INVALID_TENANT_ID;
|
tenant_id_ = common::OB_INVALID_TENANT_ID;
|
||||||
ls_id_.reset();
|
ls_id_.reset();
|
||||||
|
source_table_id_ = common::OB_INVALID_ID;
|
||||||
|
dest_table_id_ = common::OB_INVALID_ID;
|
||||||
source_tablet_id_.reset();
|
source_tablet_id_.reset();
|
||||||
dest_tablet_id_.reset();
|
dest_tablet_id_.reset();
|
||||||
if (nullptr != data_table_schema_) {
|
|
||||||
data_table_schema_->~ObTableSchema();
|
|
||||||
}
|
|
||||||
if (nullptr != hidden_table_schema_) {
|
|
||||||
hidden_table_schema_->~ObTableSchema();
|
|
||||||
}
|
|
||||||
data_table_schema_ = nullptr;
|
|
||||||
hidden_table_schema_ = nullptr;
|
|
||||||
ranges_.reset();
|
ranges_.reset();
|
||||||
allocator_.reset();
|
allocator_.reset();
|
||||||
row_store_type_ = common::ENCODING_ROW_STORE;
|
row_store_type_ = common::ENCODING_ROW_STORE;
|
||||||
@ -84,18 +78,18 @@ public:
|
|||||||
compat_mode_ = lib::Worker::CompatMode::INVALID;
|
compat_mode_ = lib::Worker::CompatMode::INVALID;
|
||||||
cluster_version_ = 0;
|
cluster_version_ = 0;
|
||||||
}
|
}
|
||||||
TO_STRING_KV(K_(is_inited), K_(tenant_id), K_(ls_id), K_(source_tablet_id), K_(dest_tablet_id),
|
TO_STRING_KV(K_(is_inited), K_(tenant_id), K_(ls_id), K_(source_table_id), K_(dest_table_id),
|
||||||
KPC_(data_table_schema), KPC_(hidden_table_schema), K_(schema_version), K_(tablet_task_id),
|
K_(source_tablet_id), K_(dest_tablet_id), K_(schema_version), K_(tablet_task_id),
|
||||||
K_(snapshot_version), K_(concurrent_cnt), K_(task_id), K_(execution_id), K_(compat_mode),
|
K_(snapshot_version), K_(concurrent_cnt), K_(task_id), K_(execution_id), K_(compat_mode),
|
||||||
K_(cluster_version));
|
K_(cluster_version));
|
||||||
public:
|
public:
|
||||||
bool is_inited_;
|
bool is_inited_;
|
||||||
uint64_t tenant_id_;
|
uint64_t tenant_id_;
|
||||||
share::ObLSID ls_id_;
|
share::ObLSID ls_id_;
|
||||||
|
uint64_t source_table_id_;
|
||||||
|
uint64_t dest_table_id_;
|
||||||
ObTabletID source_tablet_id_;
|
ObTabletID source_tablet_id_;
|
||||||
ObTabletID dest_tablet_id_;
|
ObTabletID dest_tablet_id_;
|
||||||
const share::schema::ObTableSchema *data_table_schema_;
|
|
||||||
const share::schema::ObTableSchema *hidden_table_schema_;
|
|
||||||
common::ObArenaAllocator allocator_;
|
common::ObArenaAllocator allocator_;
|
||||||
common::ObRowStoreType row_store_type_;
|
common::ObRowStoreType row_store_type_;
|
||||||
int64_t schema_version_;
|
int64_t schema_version_;
|
||||||
@ -189,12 +183,11 @@ public:
|
|||||||
int process() override;
|
int process() override;
|
||||||
private:
|
private:
|
||||||
int generate_next_task(share::ObITask *&next_task);
|
int generate_next_task(share::ObITask *&next_task);
|
||||||
int generate_col_param(const share::schema::ObTableSchema *data_table_schema,
|
int generate_col_param();
|
||||||
const share::schema::ObTableSchema *hidden_table_schema);
|
|
||||||
int local_scan_by_range();
|
int local_scan_by_range();
|
||||||
int do_local_scan();
|
int do_local_scan();
|
||||||
int append_row(ObLocalScan &local_scan);
|
int append_row(ObLocalScan &local_scan);
|
||||||
int add_extra_rowkey(const int64_t extra_rowkey_cnt, const blocksstable::ObDatumRow &row);
|
int add_extra_rowkey(const int64_t rowkey_cnt, const int64_t extra_rowkey_cnt, const blocksstable::ObDatumRow &row);
|
||||||
|
|
||||||
private:
|
private:
|
||||||
static const int64_t RETRY_INTERVAL = 100 * 1000; // 100ms
|
static const int64_t RETRY_INTERVAL = 100 * 1000; // 100ms
|
||||||
@ -250,26 +243,39 @@ public:
|
|||||||
int init(const common::ObIArray<share::schema::ObColDesc> &col_ids,
|
int init(const common::ObIArray<share::schema::ObColDesc> &col_ids,
|
||||||
const common::ObIArray<share::schema::ObColDesc> &org_col_ids,
|
const common::ObIArray<share::schema::ObColDesc> &org_col_ids,
|
||||||
const common::ObIArray<int32_t> &projector,
|
const common::ObIArray<int32_t> &projector,
|
||||||
const share::schema::ObTableSchema *data_table_schema,
|
const share::schema::ObTableSchema &data_table_schema,
|
||||||
const int64_t snapshot_version,
|
const int64_t snapshot_version,
|
||||||
transaction::ObTransService *txs,
|
transaction::ObTransService *txs,
|
||||||
const share::schema::ObTableSchema *hidden_table_schema,
|
const share::schema::ObTableSchema &hidden_table_schema,
|
||||||
const bool output_org_cols_only);
|
const bool output_org_cols_only);
|
||||||
int table_scan(const share::ObLSID &ls_id,
|
int table_scan(const share::schema::ObTableSchema &data_table_schema,
|
||||||
|
const share::ObLSID &ls_id,
|
||||||
const ObTabletID &tablet_id,
|
const ObTabletID &tablet_id,
|
||||||
ObTabletTableIterator &table_iter,
|
ObTabletTableIterator &table_iter,
|
||||||
common::ObQueryFlag &query_flag,
|
common::ObQueryFlag &query_flag,
|
||||||
blocksstable::ObDatumRange &range,
|
blocksstable::ObDatumRange &range,
|
||||||
transaction::ObTxDesc *tx_desc);
|
transaction::ObTxDesc *tx_desc);
|
||||||
virtual int get_next_row(const blocksstable::ObDatumRow *&tmp_row) override;
|
virtual int get_next_row(const blocksstable::ObDatumRow *&tmp_row) override;
|
||||||
int get_origin_table_checksum(ObArray<int64_t> &report_col_checksums, ObArray<int64_t> &report_col_ids);
|
int get_origin_table_checksum(
|
||||||
|
ObArray<int64_t> &report_col_checksums,
|
||||||
|
ObArray<int64_t> &report_col_ids);
|
||||||
compaction::ObColumnChecksumCalculator *get_checksum_calculator() {return &checksum_calculator_;}
|
compaction::ObColumnChecksumCalculator *get_checksum_calculator() {return &checksum_calculator_;}
|
||||||
private:
|
private:
|
||||||
int get_output_columns(common::ObIArray<ObColDesc> &col_ids);
|
int get_output_columns(
|
||||||
int get_exist_column_mapping(); // to record data table columns position in hidden tables.
|
const share::schema::ObTableSchema &hidden_table_schema,
|
||||||
int check_generated_column_exist(const common::ObIArray<share::schema::ObColDesc> &org_col_ids);
|
common::ObIArray<ObColDesc> &col_ids);
|
||||||
int construct_column_schema();
|
int get_exist_column_mapping(
|
||||||
int construct_access_param(const ObTabletID &tablet_id, const ObTableReadInfo &full_read_info);
|
const share::schema::ObTableSchema &data_table_schema,
|
||||||
|
const share::schema::ObTableSchema &hidden_table_schema); // to record data table columns position in hidden tables.
|
||||||
|
int check_generated_column_exist(
|
||||||
|
const share::schema::ObTableSchema &hidden_table_schema,
|
||||||
|
const common::ObIArray<share::schema::ObColDesc> &org_col_ids);
|
||||||
|
int construct_column_schema(
|
||||||
|
const share::schema::ObTableSchema &data_table_schema);
|
||||||
|
int construct_access_param(
|
||||||
|
const share::schema::ObTableSchema &data_table_schema,
|
||||||
|
const ObTabletID &tablet_id,
|
||||||
|
const ObTableReadInfo &full_read_info);
|
||||||
int construct_range_ctx(common::ObQueryFlag &query_flag, const share::ObLSID &ls_id, transaction::ObTxDesc *tx_desc);
|
int construct_range_ctx(common::ObQueryFlag &query_flag, const share::ObLSID &ls_id, transaction::ObTxDesc *tx_desc);
|
||||||
int construct_multiple_scan_merge(ObTablet &tablet, blocksstable::ObDatumRange &range);
|
int construct_multiple_scan_merge(ObTablet &tablet, blocksstable::ObDatumRange &range);
|
||||||
int construct_multiple_scan_merge(
|
int construct_multiple_scan_merge(
|
||||||
@ -277,9 +283,11 @@ private:
|
|||||||
blocksstable::ObDatumRange &range);
|
blocksstable::ObDatumRange &range);
|
||||||
private:
|
private:
|
||||||
bool is_inited_;
|
bool is_inited_;
|
||||||
|
uint64_t tenant_id_;
|
||||||
|
uint64_t source_table_id_;
|
||||||
|
uint64_t dest_table_id_;
|
||||||
|
int64_t schema_version_;
|
||||||
ObExtendedGCParam extended_gc_;
|
ObExtendedGCParam extended_gc_;
|
||||||
const share::schema::ObTableSchema *data_table_schema_;
|
|
||||||
const share::schema::ObTableSchema *hidden_table_schema_;
|
|
||||||
int64_t snapshot_version_;
|
int64_t snapshot_version_;
|
||||||
transaction::ObTransService *txs_;
|
transaction::ObTransService *txs_;
|
||||||
blocksstable::ObDatumRow default_row_;
|
blocksstable::ObDatumRow default_row_;
|
||||||
|
|||||||
Reference in New Issue
Block a user