Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
134 changes: 117 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,14 @@
#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 "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 {
class RuntimeState;
Expand Down Expand Up @@ -63,9 +67,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 +77,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));
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why remove these logic

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

user and user_ip params are deprecated in TGetDbsParams so removed it.

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 +93,122 @@ 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++) {
if ((_s_tbls_columns[j].type == TYPE_BIGINT) || _s_tbls_columns[j].type == TYPE_INT) {
SchemaScannerHelper::insert_int_value(j, row.column_value[j].longVal,
_partitions_block.get());
} else 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;
SchemaScannerHelper::insert_datetime_value(j, datas, _partitions_block.get());
} else {
SchemaScannerHelper::insert_string_value(j, row.column_value[j].stringVal,
_partitions_block.get());
}
}
}
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
62 changes: 62 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,62 @@
// 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 "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);
}
} // namespace doris
42 changes: 42 additions & 0 deletions be/src/exec/schema_scanner/schema_scanner_helper.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
// 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.

#ifndef _SCHEMA_SCANNER_HELPER_H_

#include <stdint.h>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

warning: inclusion of deprecated C++ header 'stdint.h'; consider using 'cstdint' instead [modernize-deprecated-headers]

Suggested change
#include <stdint.h>
#include <cstdint>


#include <string>
#include <vector>

// this is a util class which can be used by all shema scanner
// all common functions are added in this class.
namespace doris {

namespace vectorized {
class Block;
} // namespace vectorized
class SchemaScannerHelper {
public:
static void insert_string_value(int col_index, std::string str_val, vectorized::Block* block);
static void insert_datetime_value(int col_index, const std::vector<void*>& datas,
vectorized::Block* block);

static void insert_int_value(int col_index, int64_t int_val, vectorized::Block* block);
};

} // namespace doris
#endif
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,10 @@ public String getItemsString() {
return toString();
}

public String getItemsSql() {
return toSql();
}

@Override
public boolean isDefaultPartition() {
return isDefaultPartition;
Expand Down
19 changes: 19 additions & 0 deletions fe/fe-core/src/main/java/org/apache/doris/catalog/Partition.java
Original file line number Diff line number Diff line change
Expand Up @@ -450,4 +450,23 @@ public void convertHashDistributionToRandomDistribution() {
public boolean isRollupIndex(long id) {
return idToVisibleRollupIndex.containsKey(id);
}


public long getRowCount() {
return getBaseIndex().getRowCount();
}

public long getAvgRowLength() {
long rowCount = getBaseIndex().getRowCount();
long dataSize = getBaseIndex().getDataSize(false);
if (rowCount > 0) {
return dataSize / rowCount;
} else {
return 0;
}
}

public long getDataLength() {
return getBaseIndex().getDataSize(false);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -128,6 +128,19 @@ public List<Column> getPartitionColumns() {
return partitionColumns;
}

public String getDisplayPartitionColumns() {
StringBuilder sb = new StringBuilder();
int index = 0;
for (Column c : partitionColumns) {
if (index != 0) {
sb.append(", ");
}
sb.append(c.getDisplayName());
index++;
}
return sb.toString();
}

public Map<Long, PartitionItem> getIdToItem(boolean isTemp) {
if (isTemp) {
return idToTempItem;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,4 +62,8 @@ public boolean isDefaultPartition() {
public abstract boolean isGreaterThanSpecifiedTime(int pos, Optional<String> dateFormatOptional,
long nowTruncSubSec)
throws AnalysisException;


//get the unique string of the partition item in sql format
public abstract String getItemsSql();
}
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,10 @@ public String getItemsString() {
return toString();
}

public String getItemsSql() {
return toPartitionKeyDesc().toSql();
}

@Override
public boolean isDefaultPartition() {
return false;
Expand Down
Loading