Skip to content

Commit

Permalink
[Branch 2.1] backport systable PR (#34384,#40153,#40456,#40455,#40568) (
Browse files Browse the repository at this point in the history
#40687)

backport
#40568
#40455
#40456
#40153
#34384

Test result:
2024-09-11 11:00:45.618 INFO [suite-thread-1] (SuiteContext.groovy:309)
- Recover original connection
2024-09-11 11:00:45.619 INFO [suite-thread-1] (Suite.groovy:359) -
Execute sql: REVOKE SELECT_PRIV ON
test_partitions_schema_db.duplicate_table FROM partitions_user
2024-09-11 11:00:45.625 INFO [suite-thread-1] (SuiteContext.groovy:299)
- Create new connection for user 'partitions_user'
2024-09-11 11:00:45.632 INFO [suite-thread-1] (Suite.groovy:1162) -
Execute tag: select_check_5, sql: select
TABLE_CATALOG,TABLE_SCHEMA,TABLE_NAME,PARTITION_NAME,SUBPARTITION_NAME,PARTITION_ORDINAL_POSITION,SUBPARTITION_ORDINAL_POSITION,PARTITION_METHOD,SUBPARTITION_METHOD,PARTITION_EXPRESSION,SUBPARTITION_EXPRESSION,PARTITION_DESCRIPTION,TABLE_ROWS,AVG_ROW_LENGTH,DATA_LENGTH,MAX_DATA_LENGTH,INDEX_LENGTH,DATA_FREE,CHECKSUM,PARTITION_COMMENT,NODEGROUP,TABLESPACE_NAME
from information_schema.partitions where
table_schema="test_partitions_schema_db" order by
TABLE_CATALOG,TABLE_SCHEMA,TABLE_NAME,PARTITION_NAME,SUBPARTITION_NAME,PARTITION_ORDINAL_POSITION,SUBPARTITION_ORDINAL_POSITION,PARTITION_METHOD,SUBPARTITION_METHOD,PARTITION_EXPRESSION,SUBPARTITION_EXPRESSION,PARTITION_DESCRIPTION,TABLE_ROWS,AVG_ROW_LENGTH,DATA_LENGTH,MAX_DATA_LENGTH,INDEX_LENGTH,DATA_FREE,CHECKSUM,PARTITION_COMMENT,NODEGROUP,TABLESPACE_NAME
2024-09-11 11:00:45.644 INFO [suite-thread-1] (SuiteContext.groovy:309)
- Recover original connection
2024-09-11 11:00:45.645 INFO [suite-thread-1] (ScriptContext.groovy:120)
- Run test_partitions_schema in
/root/doris/workspace/doris/regression-test/suites/query_p0/system/test_partitions_schema.groovy
succeed
2024-09-11 11:00:45.652 INFO [main] (RegressionTest.groovy:259) - Start
to run single scripts
2024-09-11 11:01:10.321 INFO [main] (RegressionTest.groovy:380) -
Success suites:

/root/doris/workspace/doris/regression-test/suites/query_p0/system/test_partitions_schema.groovy:
group=default,p0, name=test_partitions_schema
2024-09-11 11:01:10.322 INFO [main] (RegressionTest.groovy:459) - All
suites success.
 ____   _    ____ ____  _____ ____
|  _ \ / \  / ___/ ___|| ____|  _ \
| |_) / _ \ \___ \___ \|  _| | | | |
|  __/ ___ \ ___) |__) | |___| |_| |
|_| /_/   \_\____/____/|_____|____/

2024-09-11 11:01:10.322 INFO [main] (RegressionTest.groovy:410) - Test 1
suites, failed 0 suites, fatal 0 scripts, skipped 0 scripts
2024-09-11 11:01:10.322 INFO [main] (RegressionTest.groovy:119) - Test
finished


2024-09-11 11:03:00.712 INFO [suite-thread-1] (Suite.groovy:1162) -
Execute tag: select_check_5, sql: select * from
information_schema.table_options ORDER BY
TABLE_CATALOG,TABLE_SCHEMA,TABLE_NAME,TABLE_MODEL,TABLE_MODEL_KEY,DISTRIBUTE_KEY,DISTRIBUTE_TYPE,BUCKETS_NUM,PARTITION_NUM;
2024-09-11 11:03:00.729 INFO [suite-thread-1] (SuiteContext.groovy:309)
- Recover original connection
2024-09-11 11:03:00.731 INFO [suite-thread-1] (ScriptContext.groovy:120)
- Run test_table_options in
/root/doris/workspace/doris/regression-test/suites/query_p0/system/test_table_options.groovy
succeed
2024-09-11 11:03:04.817 INFO [main] (RegressionTest.groovy:259) - Start
to run single scripts
2024-09-11 11:03:28.741 INFO [main] (RegressionTest.groovy:380) -
Success suites:

/root/doris/workspace/doris/regression-test/suites/query_p0/system/test_table_options.groovy:
group=default,p0, name=test_table_options
2024-09-11 11:03:28.742 INFO [main] (RegressionTest.groovy:459) - All
suites success.
 ____   _    ____ ____  _____ ____
|  _ \ / \  / ___/ ___|| ____|  _ \
| |_) / _ \ \___ \___ \|  _| | | | |
|  __/ ___ \ ___) |__) | |___| |_| |
|_| /_/   \_\____/____/|_____|____/

2024-09-11 11:03:28.742 INFO [main] (RegressionTest.groovy:410) - Test 1
suites, failed 0 suites, fatal 0 scripts, skipped 0 scripts
2024-09-11 11:03:28.742 INFO [main] (RegressionTest.groovy:119) - Test
finished


*************************** 7. row ***************************
             PartitionId: 18035
           PartitionName: p100
          VisibleVersion: 2
      VisibleVersionTime: 2024-09-11 10:59:28
                   State: NORMAL
            PartitionKey: col_1
Range: [types: [INT]; keys: [83647]; ..types: [INT]; keys: [2147483647];
)
         DistributionKey: pk
                 Buckets: 10
          ReplicationNum: 1
           StorageMedium: HDD
            CooldownTime: 9999-12-31 15:59:59
     RemoteStoragePolicy: 
LastConsistencyCheckTime: NULL
                DataSize: 2.872 KB
              IsInMemory: false
       ReplicaAllocation: tag.location.default: 1
               IsMutable: true
      SyncWithBaseTables: true
            UnsyncTables: NULL
        CommittedVersion: 2
                RowCount: 4
7 rows in set (0.01 sec)

---------

Co-authored-by: Mingyu Chen <[email protected]>
  • Loading branch information
Vallishp and morningman authored Sep 12, 2024
1 parent 361a59d commit 3604d63
Show file tree
Hide file tree
Showing 30 changed files with 1,360 additions and 157 deletions.
14 changes: 14 additions & 0 deletions be/src/exec/schema_scanner.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
#include "exec/schema_scanner/schema_rowsets_scanner.h"
#include "exec/schema_scanner/schema_schema_privileges_scanner.h"
#include "exec/schema_scanner/schema_schemata_scanner.h"
#include "exec/schema_scanner/schema_table_options_scanner.h"
#include "exec/schema_scanner/schema_table_privileges_scanner.h"
#include "exec/schema_scanner/schema_table_properties_scanner.h"
#include "exec/schema_scanner/schema_tables_scanner.h"
Expand Down Expand Up @@ -239,6 +240,8 @@ std::unique_ptr<SchemaScanner> SchemaScanner::create(TSchemaTableType::type type
return SchemaTablePropertiesScanner::create_unique();
case TSchemaTableType::SCH_CATALOG_META_CACHE_STATISTICS:
return SchemaCatalogMetaCacheStatsScanner::create_unique();
case TSchemaTableType::SCH_TABLE_OPTIONS:
return SchemaTableOptionsScanner::create_unique();
default:
return SchemaDummyScanner::create_unique();
break;
Expand Down Expand Up @@ -449,6 +452,17 @@ Status SchemaScanner::insert_block_column(TCell cell, int col_index, vectorized:
break;
}

case TYPE_DATETIME: {
std::vector<void*> datas(1);
VecDateTimeValue src[1];
src[0].from_date_str(cell.stringVal.data(), cell.stringVal.size());
datas[0] = src;
auto data = datas[0];
reinterpret_cast<vectorized::ColumnVector<vectorized::Int64>*>(col_ptr)->insert_data(
reinterpret_cast<char*>(data), 0);
nullable_column->get_null_map_data().emplace_back(0);
break;
}
default: {
std::stringstream ss;
ss << "unsupported column type:" << type;
Expand Down
37 changes: 4 additions & 33 deletions be/src/exec/schema_scanner/schema_active_queries_scanner.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -98,41 +98,12 @@ Status SchemaActiveQueriesScanner::_get_active_queries_block_from_fe() {
}
}

// todo(wb) reuse this callback function
auto insert_string_value = [&](int col_index, std::string str_val, vectorized::Block* block) {
vectorized::MutableColumnPtr mutable_col_ptr;
mutable_col_ptr = std::move(*block->get_by_position(col_index).column).assume_mutable();
auto* nullable_column =
reinterpret_cast<vectorized::ColumnNullable*>(mutable_col_ptr.get());
vectorized::IColumn* col_ptr = &nullable_column->get_nested_column();
reinterpret_cast<vectorized::ColumnString*>(col_ptr)->insert_data(str_val.data(),
str_val.size());
nullable_column->get_null_map_data().emplace_back(0);
};
auto insert_int_value = [&](int col_index, int64_t int_val, vectorized::Block* block) {
vectorized::MutableColumnPtr mutable_col_ptr;
mutable_col_ptr = std::move(*block->get_by_position(col_index).column).assume_mutable();
auto* nullable_column =
reinterpret_cast<vectorized::ColumnNullable*>(mutable_col_ptr.get());
vectorized::IColumn* col_ptr = &nullable_column->get_nested_column();
reinterpret_cast<vectorized::ColumnVector<vectorized::Int64>*>(col_ptr)->insert_value(
int_val);
nullable_column->get_null_map_data().emplace_back(0);
};

for (int i = 0; i < result_data.size(); i++) {
TRow row = result_data[i];

insert_string_value(0, row.column_value[0].stringVal, _active_query_block.get());
insert_string_value(1, row.column_value[1].stringVal, _active_query_block.get());
insert_int_value(2, row.column_value[2].longVal, _active_query_block.get());
insert_int_value(3, row.column_value[3].longVal, _active_query_block.get());
insert_string_value(4, row.column_value[4].stringVal, _active_query_block.get());
insert_string_value(5, row.column_value[5].stringVal, _active_query_block.get());
insert_string_value(6, row.column_value[6].stringVal, _active_query_block.get());
insert_string_value(7, row.column_value[7].stringVal, _active_query_block.get());
insert_string_value(8, row.column_value[8].stringVal, _active_query_block.get());
insert_string_value(9, row.column_value[9].stringVal, _active_query_block.get());
for (int j = 0; j < _s_tbls_columns.size(); j++) {
RETURN_IF_ERROR(insert_block_column(row.column_value[j], j, _active_query_block.get(),
_s_tbls_columns[j].type));
}
}
return Status::OK();
}
Expand Down
120 changes: 103 additions & 17 deletions be/src/exec/schema_scanner/schema_partitions_scanner.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -22,10 +22,13 @@
#include <stdint.h>

#include "exec/schema_scanner/schema_helper.h"
#include "runtime/decimalv2_value.h"
#include "runtime/define_primitive_type.h"
#include "util/runtime_profile.h"
#include "runtime/client_cache.h"
#include "runtime/exec_env.h"
#include "runtime/runtime_state.h"
#include "util/thrift_rpc_helper.h"
#include "vec/common/string_ref.h"
#include "vec/core/block.h"
#include "vec/data_types/data_type_factory.hpp"

namespace doris {
class RuntimeState;
Expand Down Expand Up @@ -63,9 +66,7 @@ std::vector<SchemaScanner::ColumnDesc> SchemaPartitionsScanner::_s_tbls_columns
};

SchemaPartitionsScanner::SchemaPartitionsScanner()
: SchemaScanner(_s_tbls_columns, TSchemaTableType::SCH_PARTITIONS),
_db_index(0),
_table_index(0) {}
: SchemaScanner(_s_tbls_columns, TSchemaTableType::SCH_PARTITIONS) {}

SchemaPartitionsScanner::~SchemaPartitionsScanner() {}

Expand All @@ -75,21 +76,14 @@ Status SchemaPartitionsScanner::start(RuntimeState* state) {
}
SCOPED_TIMER(_get_db_timer);
TGetDbsParams db_params;
if (nullptr != _param->common_param->db) {
if (_param->common_param->db) {
db_params.__set_pattern(*(_param->common_param->db));
}
if (nullptr != _param->common_param->catalog) {
if (_param->common_param->catalog) {
db_params.__set_catalog(*(_param->common_param->catalog));
}
if (nullptr != _param->common_param->current_user_ident) {
if (_param->common_param->current_user_ident) {
db_params.__set_current_user_ident(*(_param->common_param->current_user_ident));
} else {
if (nullptr != _param->common_param->user) {
db_params.__set_user(*(_param->common_param->user));
}
if (nullptr != _param->common_param->user_ip) {
db_params.__set_user_ip(*(_param->common_param->user_ip));
}
}

if (nullptr != _param->common_param->ip && 0 != _param->common_param->port) {
Expand All @@ -98,17 +92,109 @@ Status SchemaPartitionsScanner::start(RuntimeState* state) {
} else {
return Status::InternalError("IP or port doesn't exists");
}
_block_rows_limit = state->batch_size();
_rpc_timeout_ms = state->execution_timeout() * 1000;
return Status::OK();
}

Status SchemaPartitionsScanner::get_onedb_info_from_fe(int64_t dbId) {
TNetworkAddress master_addr = ExecEnv::GetInstance()->master_info()->network_address;

TSchemaTableRequestParams schema_table_request_params;
for (int i = 0; i < _s_tbls_columns.size(); i++) {
schema_table_request_params.__isset.columns_name = true;
schema_table_request_params.columns_name.emplace_back(_s_tbls_columns[i].name);
}
schema_table_request_params.__set_current_user_ident(*_param->common_param->current_user_ident);
schema_table_request_params.__set_catalog(*_param->common_param->catalog);
schema_table_request_params.__set_dbId(dbId);

TFetchSchemaTableDataRequest request;
request.__set_schema_table_name(TSchemaTableName::PARTITIONS);
request.__set_schema_table_params(schema_table_request_params);

TFetchSchemaTableDataResult result;

RETURN_IF_ERROR(ThriftRpcHelper::rpc<FrontendServiceClient>(
master_addr.hostname, master_addr.port,
[&request, &result](FrontendServiceConnection& client) {
client->fetchSchemaTableData(result, request);
},
_rpc_timeout_ms));

Status status(Status::create(result.status));
if (!status.ok()) {
LOG(WARNING) << "fetch table options from FE failed, errmsg=" << status;
return status;
}
std::vector<TRow> result_data = result.data_batch;

_partitions_block = vectorized::Block::create_unique();
for (int i = 0; i < _s_tbls_columns.size(); ++i) {
TypeDescriptor descriptor(_s_tbls_columns[i].type);
auto data_type = vectorized::DataTypeFactory::instance().create_data_type(descriptor, true);
_partitions_block->insert(vectorized::ColumnWithTypeAndName(
data_type->create_column(), data_type, _s_tbls_columns[i].name));
}
_partitions_block->reserve(_block_rows_limit);
if (result_data.size() > 0) {
int col_size = result_data[0].column_value.size();
if (col_size != _s_tbls_columns.size()) {
return Status::InternalError<false>("table options schema is not match for FE and BE");
}
}

for (int i = 0; i < result_data.size(); i++) {
TRow row = result_data[i];
for (int j = 0; j < _s_tbls_columns.size(); j++) {
RETURN_IF_ERROR(insert_block_column(row.column_value[j], j, _partitions_block.get(),
_s_tbls_columns[j].type));
}
}
return Status::OK();
}

bool SchemaPartitionsScanner::check_and_mark_eos(bool* eos) const {
if (_row_idx == _total_rows) {
*eos = true;
if (_db_index < _db_result.db_ids.size()) {
*eos = false;
}
return true;
}
return false;
}

Status SchemaPartitionsScanner::get_next_block_internal(vectorized::Block* block, bool* eos) {
if (!_is_init) {
return Status::InternalError("Used before initialized.");
}

if (nullptr == block || nullptr == eos) {
return Status::InternalError("input pointer is nullptr.");
}
*eos = true;

if ((_partitions_block == nullptr) || (_row_idx == _total_rows)) {
if (_db_index < _db_result.db_ids.size()) {
RETURN_IF_ERROR(get_onedb_info_from_fe(_db_result.db_ids[_db_index]));
_row_idx = 0; // reset row index so that it start filling for next block.
_total_rows = _partitions_block->rows();
_db_index++;
}
}

if (check_and_mark_eos(eos)) {
return Status::OK();
}

int current_batch_rows = std::min(_block_rows_limit, _total_rows - _row_idx);
vectorized::MutableBlock mblock = vectorized::MutableBlock::build_mutable_block(block);
RETURN_IF_ERROR(mblock.add_rows(_partitions_block.get(), _row_idx, current_batch_rows));
_row_idx += current_batch_rows;

if (!check_and_mark_eos(eos)) {
*eos = false;
}
return Status::OK();
}

Expand Down
15 changes: 11 additions & 4 deletions be/src/exec/schema_scanner/schema_partitions_scanner.h
Original file line number Diff line number Diff line change
Expand Up @@ -40,11 +40,18 @@ class SchemaPartitionsScanner : public SchemaScanner {
Status start(RuntimeState* state) override;
Status get_next_block_internal(vectorized::Block* block, bool* eos) override;

int _db_index;
int _table_index;
TGetDbsResult _db_result;
TListTableStatusResult _table_result;
static std::vector<SchemaScanner::ColumnDesc> _s_tbls_columns;

private:
Status get_onedb_info_from_fe(int64_t dbId);
bool check_and_mark_eos(bool* eos) const;
int _block_rows_limit = 4096;
int _db_index = 0;
TGetDbsResult _db_result;
int _row_idx = 0;
int _total_rows = 0;
std::unique_ptr<vectorized::Block> _partitions_block = nullptr;
int _rpc_timeout_ms = 3000;
};

} // namespace doris
35 changes: 2 additions & 33 deletions be/src/exec/schema_scanner/schema_routine_scanner.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -99,43 +99,12 @@ Status SchemaRoutinesScanner::get_block_from_fe() {
return Status::InternalError<false>("routine table schema is not match for FE and BE");
}
}
auto insert_string_value = [&](int col_index, std::string str_val, vectorized::Block* block) {
vectorized::MutableColumnPtr mutable_col_ptr;
mutable_col_ptr = std::move(*block->get_by_position(col_index).column).assume_mutable();
auto* nullable_column =
reinterpret_cast<vectorized::ColumnNullable*>(mutable_col_ptr.get());
vectorized::IColumn* col_ptr = &nullable_column->get_nested_column();
reinterpret_cast<vectorized::ColumnString*>(col_ptr)->insert_data(str_val.data(),
str_val.size());
nullable_column->get_null_map_data().emplace_back(0);
};
auto insert_datetime_value = [&](int col_index, const std::vector<void*>& datas,
vectorized::Block* block) {
vectorized::MutableColumnPtr mutable_col_ptr;
mutable_col_ptr = std::move(*block->get_by_position(col_index).column).assume_mutable();
auto* nullable_column =
reinterpret_cast<vectorized::ColumnNullable*>(mutable_col_ptr.get());
vectorized::IColumn* col_ptr = &nullable_column->get_nested_column();
auto data = datas[0];
reinterpret_cast<vectorized::ColumnVector<vectorized::Int64>*>(col_ptr)->insert_data(
reinterpret_cast<char*>(data), 0);
nullable_column->get_null_map_data().emplace_back(0);
};

for (int i = 0; i < result_data.size(); i++) {
TRow row = result_data[i];

for (int j = 0; j < _s_tbls_columns.size(); j++) {
if (_s_tbls_columns[j].type == TYPE_DATETIME) {
std::vector<void*> datas(1);
VecDateTimeValue src[1];
src[0].from_date_str(row.column_value[j].stringVal.data(),
row.column_value[j].stringVal.size());
datas[0] = src;
insert_datetime_value(j, datas, _routines_block.get());
} else {
insert_string_value(j, row.column_value[j].stringVal, _routines_block.get());
}
RETURN_IF_ERROR(insert_block_column(row.column_value[j], j, _routines_block.get(),
_s_tbls_columns[j].type));
}
}
return Status::OK();
Expand Down
73 changes: 73 additions & 0 deletions be/src/exec/schema_scanner/schema_scanner_helper.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

#include "exec/schema_scanner/schema_scanner_helper.h"

#include "runtime/client_cache.h"
#include "runtime/exec_env.h"
#include "runtime/runtime_state.h"
#include "util/thrift_rpc_helper.h"
#include "vec/common/string_ref.h"
#include "vec/core/block.h"
#include "vec/data_types/data_type_factory.hpp"

namespace doris {

void SchemaScannerHelper::insert_string_value(int col_index, std::string str_val,
vectorized::Block* block) {
vectorized::MutableColumnPtr mutable_col_ptr;
mutable_col_ptr = std::move(*block->get_by_position(col_index).column).assume_mutable();
auto* nullable_column = reinterpret_cast<vectorized::ColumnNullable*>(mutable_col_ptr.get());
vectorized::IColumn* col_ptr = &nullable_column->get_nested_column();
reinterpret_cast<vectorized::ColumnString*>(col_ptr)->insert_data(str_val.data(),
str_val.size());
nullable_column->get_null_map_data().emplace_back(0);
}

void SchemaScannerHelper::insert_datetime_value(int col_index, const std::vector<void*>& datas,
vectorized::Block* block) {
vectorized::MutableColumnPtr mutable_col_ptr;
mutable_col_ptr = std::move(*block->get_by_position(col_index).column).assume_mutable();
auto* nullable_column = reinterpret_cast<vectorized::ColumnNullable*>(mutable_col_ptr.get());
vectorized::IColumn* col_ptr = &nullable_column->get_nested_column();
auto data = datas[0];
reinterpret_cast<vectorized::ColumnVector<vectorized::Int64>*>(col_ptr)->insert_data(
reinterpret_cast<char*>(data), 0);
nullable_column->get_null_map_data().emplace_back(0);
}

void SchemaScannerHelper::insert_int_value(int col_index, int64_t int_val,
vectorized::Block* block) {
vectorized::MutableColumnPtr mutable_col_ptr;
mutable_col_ptr = std::move(*block->get_by_position(col_index).column).assume_mutable();
auto* nullable_column = reinterpret_cast<vectorized::ColumnNullable*>(mutable_col_ptr.get());
vectorized::IColumn* col_ptr = &nullable_column->get_nested_column();
reinterpret_cast<vectorized::ColumnVector<vectorized::Int64>*>(col_ptr)->insert_value(int_val);
nullable_column->get_null_map_data().emplace_back(0);
}

void SchemaScannerHelper::insert_double_value(int col_index, double double_val,
vectorized::Block* block) {
vectorized::MutableColumnPtr mutable_col_ptr;
mutable_col_ptr = std::move(*block->get_by_position(col_index).column).assume_mutable();
auto* nullable_column = reinterpret_cast<vectorized::ColumnNullable*>(mutable_col_ptr.get());
vectorized::IColumn* col_ptr = &nullable_column->get_nested_column();
reinterpret_cast<vectorized::ColumnVector<vectorized::Float64>*>(col_ptr)->insert_value(
double_val);
nullable_column->get_null_map_data().emplace_back(0);
}
} // namespace doris
Loading

0 comments on commit 3604d63

Please sign in to comment.