Skip to content

Commit

Permalink
1
Browse files Browse the repository at this point in the history
  • Loading branch information
zddr committed Mar 28, 2024
2 parents 98a7436 + fd1cf23 commit 604cfa4
Show file tree
Hide file tree
Showing 344 changed files with 9,627 additions and 3,617 deletions.
3 changes: 2 additions & 1 deletion be/src/agent/task_worker_pool.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -190,7 +190,8 @@ void alter_tablet(StorageEngine& engine, const TAgentTaskRequest& agent_task_req
Status res = Status::OK();
try {
DCHECK(agent_task_req.alter_tablet_req_v2.__isset.job_id);
SchemaChangeJob job(engine, agent_task_req.alter_tablet_req_v2);
SchemaChangeJob job(engine, agent_task_req.alter_tablet_req_v2,
std::to_string(agent_task_req.alter_tablet_req_v2.job_id));
status = job.process_alter_tablet(agent_task_req.alter_tablet_req_v2);
} catch (const Exception& e) {
status = e.to_status();
Expand Down
4 changes: 4 additions & 0 deletions be/src/cloud/cloud_schema_change_job.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -201,6 +201,10 @@ Status CloudSchemaChangeJob::_convert_historical_rowsets(const SchemaChangeParam
"Don't support to add materialized view by linked schema change");
}

LOG(INFO) << "schema change type, sc_sorting: " << sc_sorting
<< ", sc_directly: " << sc_directly << ", base_tablet=" << _base_tablet->tablet_id()
<< ", new_tablet=" << _new_tablet->tablet_id();

// 2. Generate historical data converter
auto sc_procedure = get_sc_procedure(changer, sc_sorting);

Expand Down
6 changes: 3 additions & 3 deletions be/src/exec/exec_node.h
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,7 @@ class ExecNode {
bool has_output_row_descriptor() const { return _output_row_descriptor != nullptr; }
// If use projection, we should clear `_origin_block`.
void clear_origin_block() {
_origin_block.clear_column_data(_row_descriptor.num_materialized_slots());
_origin_block.clear_column_data(intermediate_row_desc().num_materialized_slots());
}

// Emit data, both need impl with method: sink
Expand Down Expand Up @@ -326,8 +326,8 @@ class ExecNode {
std::shared_ptr<QueryStatistics> _query_statistics = nullptr;

//_keep_origin is used to avoid copying during projection,
// currently set to true only in the nestloop join.
bool _keep_origin = false;
// currently set to false only in the nestloop join.
bool _keep_origin = true;

private:
static Status create_tree_helper(RuntimeState* state, ObjectPool* pool,
Expand Down
3 changes: 3 additions & 0 deletions be/src/exec/schema_scanner.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
#include "exec/schema_scanner/schema_partitions_scanner.h"
#include "exec/schema_scanner/schema_processlist_scanner.h"
#include "exec/schema_scanner/schema_profiling_scanner.h"
#include "exec/schema_scanner/schema_routine_scanner.h"
#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"
Expand Down Expand Up @@ -161,6 +162,8 @@ std::unique_ptr<SchemaScanner> SchemaScanner::create(TSchemaTableType::type type
return SchemaWorkloadGroupsScanner::create_unique();
case TSchemaTableType::SCH_PROCESSLIST:
return SchemaProcessListScanner::create_unique();
case TSchemaTableType::SCH_PROCEDURES:
return SchemaRoutinesScanner::create_unique();
default:
return SchemaDummyScanner::create_unique();
break;
Expand Down
172 changes: 172 additions & 0 deletions be/src/exec/schema_scanner/schema_routine_scanner.cpp
Original file line number Diff line number Diff line change
@@ -0,0 +1,172 @@
// 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_routine_scanner.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 {
std::vector<SchemaScanner::ColumnDesc> SchemaRoutinesScanner::_s_tbls_columns = {
{"SPECIFIC_NAME", TYPE_VARCHAR, sizeof(StringRef), true},
{"ROUTINE_CATALOG", TYPE_VARCHAR, sizeof(StringRef), true},
{"ROUTINE_SCHEMA", TYPE_VARCHAR, sizeof(StringRef), true},
{"ROUTINE_NAME", TYPE_VARCHAR, sizeof(StringRef), true},
{"ROUTINE_TYPE", TYPE_VARCHAR, sizeof(StringRef), true},
{"DTD_IDENTIFIER", TYPE_VARCHAR, sizeof(StringRef), true},
{"ROUTINE_BODY", TYPE_VARCHAR, sizeof(StringRef), true},
{"ROUTINE_DEFINITION", TYPE_VARCHAR, sizeof(StringRef), true},
{"EXTERNAL_NAME", TYPE_VARCHAR, sizeof(StringRef), true},
{"EXTERNAL_LANGUAGE", TYPE_VARCHAR, sizeof(StringRef), true},
{"PARAMETER_STYLE", TYPE_VARCHAR, sizeof(StringRef), true},
{"IS_DETERMINISTIC", TYPE_VARCHAR, sizeof(StringRef), true},
{"SQL_DATA_ACCESS", TYPE_VARCHAR, sizeof(StringRef), true},
{"SQL_PATH", TYPE_VARCHAR, sizeof(StringRef), true},
{"SECURITY_TYPE", TYPE_VARCHAR, sizeof(StringRef), true},
{"CREATED", TYPE_DATETIME, sizeof(int64_t), true},
{"LAST_ALTERED", TYPE_DATETIME, sizeof(int64_t), true},
{"SQL_MODE", TYPE_VARCHAR, sizeof(StringRef), true},
{"ROUTINE_COMMENT", TYPE_VARCHAR, sizeof(StringRef), true},
{"DEFINER", TYPE_VARCHAR, sizeof(StringRef), true},
{"CHARACTER_SET_CLIENT", TYPE_VARCHAR, sizeof(StringRef), true},
{"COLLATION_CONNECTION", TYPE_VARCHAR, sizeof(StringRef), true},
{"DATABASE_COLLATION", TYPE_VARCHAR, sizeof(StringRef), true},
};

SchemaRoutinesScanner::SchemaRoutinesScanner()
: SchemaScanner(_s_tbls_columns, TSchemaTableType::SCH_PROCEDURES) {}

Status SchemaRoutinesScanner::start(RuntimeState* state) {
_block_rows_limit = state->batch_size();
_rpc_timeout = state->execution_timeout() * 1000;
return Status::OK();
}

Status SchemaRoutinesScanner::get_block_from_fe() {
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);
TFetchSchemaTableDataRequest request;
request.__set_schema_table_name(TSchemaTableName::ROUTINES_INFO);
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));
Status status(Status::create(result.status));
if (!status.ok()) {
LOG(WARNING) << "fetch routines from FE failed, errmsg=" << status;
return status;
}
std::vector<TRow> result_data = result.data_batch;
_routines_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);
_routines_block->insert(vectorized::ColumnWithTypeAndName(
data_type->create_column(), data_type, _s_tbls_columns[i].name));
}
_routines_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>("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 Status::OK();
}

Status SchemaRoutinesScanner::get_next_block(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.");
}

if (_routines_block == nullptr) {
RETURN_IF_ERROR(get_block_from_fe());
_total_rows = _routines_block->rows();
}

if (_row_idx == _total_rows) {
*eos = true;
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);
mblock.add_rows(_routines_block.get(), _row_idx, current_batch_rows);
_row_idx += current_batch_rows;

*eos = _row_idx == _total_rows;
return Status::OK();
}

} // namespace doris
52 changes: 52 additions & 0 deletions be/src/exec/schema_scanner/schema_routine_scanner.h
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
// 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.

#pragma once

#include <vector>

#include "common/status.h"
#include "exec/schema_scanner.h"

namespace doris {
class RuntimeState;
namespace vectorized {
class Block;
} // namespace vectorized

class SchemaRoutinesScanner : public SchemaScanner {
ENABLE_FACTORY_CREATOR(SchemaRoutinesScanner);

public:
SchemaRoutinesScanner();
~SchemaRoutinesScanner() override = default;

Status start(RuntimeState* state) override;
Status get_next_block(vectorized::Block* block, bool* eos) override;

static std::vector<SchemaScanner::ColumnDesc> _s_tbls_columns;

private:
Status get_block_from_fe();

int _block_rows_limit = 4096;
int _row_idx = 0;
int _total_rows = 0;
std::unique_ptr<vectorized::Block> _routines_block = nullptr;
int _rpc_timeout = 3000;
};
}; // namespace doris
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ std::vector<SchemaScanner::ColumnDesc> SchemaWorkloadGroupsScanner::_s_tbls_colu
{"MIN_REMOTE_SCAN_THREAD_NUM", TYPE_BIGINT, sizeof(int64_t), true},
{"SPILL_THRESHOLD_LOW_WATERMARK", TYPE_VARCHAR, sizeof(StringRef), true},
{"SPILL_THRESHOLD_HIGH_WATERMARK", TYPE_VARCHAR, sizeof(StringRef), true},
{"TAG", TYPE_VARCHAR, sizeof(StringRef), true},
};

SchemaWorkloadGroupsScanner::SchemaWorkloadGroupsScanner()
Expand Down
4 changes: 2 additions & 2 deletions be/src/olap/hll.h
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ class HyperLogLog {
}
}

HyperLogLog(HyperLogLog&& other) {
HyperLogLog(HyperLogLog&& other) noexcept {
this->_type = other._type;
switch (other._type) {
case HLL_DATA_EMPTY:
Expand All @@ -131,7 +131,7 @@ class HyperLogLog {
}
}

HyperLogLog& operator=(HyperLogLog&& other) {
HyperLogLog& operator=(HyperLogLog&& other) noexcept {
if (this != &other) {
if (_registers != nullptr) {
delete[] _registers;
Expand Down
Loading

0 comments on commit 604cfa4

Please sign in to comment.