Skip to content

Commit

Permalink
[refactor] replace boost smart ptr with stl (#6856)
Browse files Browse the repository at this point in the history
1. replace all boost::shared_ptr to std::shared_ptr
2. replace all boost::scopted_ptr to std::unique_ptr
3. replace all boost::scoped_array to std::unique<T[]>
4. replace all boost:thread to std::thread
  • Loading branch information
yangzhg authored Nov 17, 2021
1 parent 4bc5ba8 commit 6c63809
Show file tree
Hide file tree
Showing 383 changed files with 3,442 additions and 3,395 deletions.
9 changes: 2 additions & 7 deletions be/CMakeLists.txt
Original file line number Diff line number Diff line change
Expand Up @@ -135,9 +135,9 @@ set(BOOST_ROOT ${THIRDPARTY_DIR})
set(Boost_NO_BOOST_CMAKE OFF)

if (NOT APPLE)
find_package(Boost 1.73.0 REQUIRED COMPONENTS regex system filesystem thread date_time program_options)
find_package(Boost 1.73.0 REQUIRED COMPONENTS system date_time)
else()
find_package(Boost 1.73.0 COMPONENTS thread regex system filesystem date_time program_options)
find_package(Boost 1.73.0 COMPONENTS system date_time)
endif()

set(GPERFTOOLS_HOME "${THIRDPARTY_DIR}/gperftools")
Expand Down Expand Up @@ -482,12 +482,7 @@ set(COMMON_THIRDPARTY
librdkafka
libs2
snappy
Boost::regex
Boost::system
Boost::filesystem
Boost::thread
Boost::date_time
Boost::program_options
thrift
thriftnb
glog
Expand Down
2 changes: 1 addition & 1 deletion be/src/agent/heartbeat_server.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -174,7 +174,7 @@ AgentStatus create_heartbeat_server(ExecEnv* exec_env, uint32_t server_port,
ThriftServer** thrift_server, uint32_t worker_thread_num,
TMasterInfo* local_master_info) {
HeartbeatServer* heartbeat_server = new (nothrow) HeartbeatServer(local_master_info);
if (heartbeat_server == NULL) {
if (heartbeat_server == nullptr) {
return DORIS_ERROR;
}

Expand Down
12 changes: 6 additions & 6 deletions be/src/agent/pusher.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ AgentStatus Pusher::init() {
// Check replica exist
OLAPTablePtr olap_table;
olap_table = _engine->get_table(_push_req.tablet_id, _push_req.schema_hash);
if (olap_table.get() == NULL) {
if (olap_table.get() == nullptr) {
OLAP_LOG_WARNING("get tables failed. tablet_id: %ld, schema_hash: %ld", _push_req.tablet_id,
_push_req.schema_hash);
return DORIS_PUSH_INVALID_TABLE;
Expand Down Expand Up @@ -94,10 +94,10 @@ AgentStatus Pusher::_get_tmp_file_dir(const string& root_path, string* download_

if (!std::filesystem::exists(full_path)) {
LOG(INFO) << "download dir not exist: " << *download_path;
boost::system::error_code error_code;
std::error_code error_code;
std::filesystem::create_directories(*download_path, error_code);

if (0 != error_code) {
if (error_code) {
status = DORIS_ERROR;
LOG(WARNING) << "create download dir failed.path: " << *download_path
<< ", error code: " << error_code;
Expand Down Expand Up @@ -130,7 +130,7 @@ AgentStatus Pusher::process(vector<TTabletInfo>* tablet_infos) {
bool is_timeout = false;
auto download_cb = [this, estimate_time_out, file_size, &is_timeout](HttpClient* client) {
// Check timeout and set timeout
time_t now = time(NULL);
time_t now = time(nullptr);
if (_push_req.timeout > 0 && _push_req.timeout < now) {
// return status to break this callback
VLOG_NOTICE << "check time out. time_out:" << _push_req.timeout << ", now:" << now;
Expand Down Expand Up @@ -191,9 +191,9 @@ AgentStatus Pusher::process(vector<TTabletInfo>* tablet_infos) {

if (status == DORIS_SUCCESS) {
// Load delta file
time_t push_begin = time(NULL);
time_t push_begin = time(nullptr);
OLAPStatus push_status = _engine->push(_push_req, tablet_infos);
time_t push_finish = time(NULL);
time_t push_finish = time(nullptr);
LOG(INFO) << "Push finish, cost time: " << (push_finish - push_begin);
if (push_status == OLAPStatus::OLAP_ERR_PUSH_TRANSACTION_ALREADY_EXIST) {
status = DORIS_PUSH_HAD_LOADED;
Expand Down
11 changes: 7 additions & 4 deletions be/src/agent/task_worker_pool.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,7 @@ DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(agent_task_queue_size, MetricUnit::NOUNIT);
const uint32_t TASK_FINISH_MAX_RETRY = 3;
const uint32_t PUBLISH_VERSION_MAX_RETRY = 3;

std::atomic_ulong TaskWorkerPool::_s_report_version(time(NULL) * 10000);
std::atomic_ulong TaskWorkerPool::_s_report_version(time(nullptr) * 10000);
Mutex TaskWorkerPool::_s_task_signatures_lock;
map<TTaskType::type, set<int64_t>> TaskWorkerPool::_s_task_signatures;
FrontendServiceClientCache TaskWorkerPool::_master_service_client_cache;
Expand Down Expand Up @@ -1115,7 +1115,8 @@ void TaskWorkerPool::_report_task_worker_thread_callback() {
if (_master_info.network_address.port == 0) {
// port == 0 means not received heartbeat yet
// sleep a short time and try again
LOG(INFO) << "waiting to receive first heartbeat from frontend before doing task report";
LOG(INFO)
<< "waiting to receive first heartbeat from frontend before doing task report";
continue;
}

Expand Down Expand Up @@ -1149,7 +1150,8 @@ void TaskWorkerPool::_report_disk_state_worker_thread_callback() {

if (_master_info.network_address.port == 0) {
// port == 0 means not received heartbeat yet
LOG(INFO) << "waiting to receive first heartbeat from frontend before doing disk report";
LOG(INFO)
<< "waiting to receive first heartbeat from frontend before doing disk report";
continue;
}

Expand Down Expand Up @@ -1199,7 +1201,8 @@ void TaskWorkerPool::_report_tablet_worker_thread_callback() {

if (_master_info.network_address.port == 0) {
// port == 0 means not received heartbeat yet
LOG(INFO) << "waiting to receive first heartbeat from frontend before doing tablet report";
LOG(INFO) << "waiting to receive first heartbeat from frontend before doing tablet "
"report";
continue;
}

Expand Down
2 changes: 1 addition & 1 deletion be/src/agent/topic_subscriber.h
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,9 @@
#ifndef DORIS_BE_SRC_AGENT_TOPIC_SUBSCRIBER_H
#define DORIS_BE_SRC_AGENT_TOPIC_SUBSCRIBER_H

#include <boost/thread.hpp>
#include <map>
#include <shared_mutex>
#include <thread>

#include "agent/topic_listener.h"
#include "gen_cpp/AgentService_types.h"
Expand Down
8 changes: 4 additions & 4 deletions be/src/agent/utils.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -152,10 +152,10 @@ AgentStatus AgentUtils::rsync_from_remote(const string& remote_host, const strin
cmd_stream << " " << remote_host << ":" << remote_file_path << " " << local_file_path;
LOG(INFO) << "rsync cmd: " << cmd_stream.str();

FILE* fp = NULL;
FILE* fp = nullptr;
fp = popen(cmd_stream.str().c_str(), "r");

if (fp == NULL) {
if (fp == nullptr) {
return DORIS_ERROR;
}

Expand Down Expand Up @@ -226,7 +226,7 @@ bool AgentUtils::exec_cmd(const string& command, string* errmsg, bool redirect_s

// Execute command.
FILE* fp = popen(cmd.c_str(), "r");
if (fp == NULL) {
if (fp == nullptr) {
std::stringstream err_stream;
err_stream << "popen failed. " << strerror(errno) << ", with errno: " << errno << ".\n";
*errmsg = err_stream.str();
Expand All @@ -235,7 +235,7 @@ bool AgentUtils::exec_cmd(const string& command, string* errmsg, bool redirect_s

// Get command output.
char result[1024] = {'\0'};
while (fgets(result, sizeof(result), fp) != NULL) {
while (fgets(result, sizeof(result), fp) != nullptr) {
*errmsg += result;
}

Expand Down
6 changes: 3 additions & 3 deletions be/src/common/logconfig.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ bool init_glog(const char* basename, bool install_signal_handler) {
FLAGS_log_split_method = "size";
std::string sizestr = rollmode.substr(sizeflag.size(), rollmode.size() - sizeflag.size());
if (sizestr.size() != 0) {
char* end = NULL;
char* end = nullptr;
errno = 0;
const char* sizecstr = sizestr.c_str();
int64_t ret64 = strtoll(sizecstr, &end, 10);
Expand Down Expand Up @@ -162,8 +162,8 @@ std::string FormatTimestampForLog(MicrosecondsInt64 micros_since_epoch) {
/// Custom your log format here
void TaggableLogger::flush() {
_stream << _message;
Tags *head = _tags;
Tags *next;
Tags* head = _tags;
Tags* next;
while (head) {
next = head->next;
_stream << "|" << head->key << "=" << head->value;
Expand Down
4 changes: 2 additions & 2 deletions be/src/env/env.h
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ class Env {

// Create a brand new sequentially-readable file with the specified name.
// On success, stores a pointer to the new file in *result and returns OK.
// On failure stores NULL in *result and returns non-OK. If the file does
// On failure stores nullptr in *result and returns non-OK. If the file does
// not exist, returns a non-OK status.
//
// The returned file will only be accessed by one thread at a time.
Expand All @@ -71,7 +71,7 @@ class Env {
// Create an object that writes to a new file with the specified
// name. Deletes any existing file with the same name and creates a
// new file. On success, stores a pointer to the new file in
// *result and returns OK. On failure stores NULL in *result and
// *result and returns OK. On failure stores nullptr in *result and
// returns non-OK.
//
// The returned file will only be accessed by one thread at a time.
Expand Down
54 changes: 28 additions & 26 deletions be/src/exec/aggregation_node.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -47,17 +47,17 @@ AggregationNode::AggregationNode(ObjectPool* pool, const TPlanNode& tnode,
const DescriptorTbl& descs)
: ExecNode(pool, tnode, descs),
_intermediate_tuple_id(tnode.agg_node.intermediate_tuple_id),
_intermediate_tuple_desc(NULL),
_intermediate_tuple_desc(nullptr),
_output_tuple_id(tnode.agg_node.output_tuple_id),
_output_tuple_desc(NULL),
_singleton_output_tuple(NULL),
_output_tuple_desc(nullptr),
_singleton_output_tuple(nullptr),
//_tuple_pool(new MemPool()),
//
_process_row_batch_fn(NULL),
_process_row_batch_fn(nullptr),
_needs_finalize(tnode.agg_node.need_finalize),
_build_timer(NULL),
_get_results_timer(NULL),
_hash_table_buckets_counter(NULL) {}
_build_timer(nullptr),
_get_results_timer(nullptr),
_hash_table_buckets_counter(nullptr) {}

AggregationNode::~AggregationNode() {}

Expand All @@ -68,7 +68,7 @@ Status AggregationNode::init(const TPlanNode& tnode, RuntimeState* state) {
Expr::create_expr_trees(_pool, tnode.agg_node.grouping_exprs, &_probe_expr_ctxs));

for (int i = 0; i < tnode.agg_node.aggregate_functions.size(); ++i) {
AggFnEvaluator* evaluator = NULL;
AggFnEvaluator* evaluator = nullptr;
AggFnEvaluator::create(_pool, tnode.agg_node.aggregate_functions[i], &evaluator);
_aggregate_evaluators.push_back(evaluator);
}
Expand Down Expand Up @@ -177,9 +177,9 @@ Status AggregationNode::open(RuntimeState* state) {

int64_t agg_rows_before = _hash_tbl->size();

if (_process_row_batch_fn != NULL) {
if (_process_row_batch_fn != nullptr) {
_process_row_batch_fn(this, &batch);
} else if (_singleton_output_tuple != NULL) {
} else if (_singleton_output_tuple != nullptr) {
SCOPED_TIMER(_build_timer);
process_row_batch_no_grouping(&batch, _tuple_pool.get());
} else {
Expand Down Expand Up @@ -210,7 +210,7 @@ Status AggregationNode::open(RuntimeState* state) {
}
}

if (_singleton_output_tuple != NULL) {
if (_singleton_output_tuple != nullptr) {
_hash_tbl->insert(reinterpret_cast<TupleRow*>(&_singleton_output_tuple));
++num_agg_rows;
}
Expand All @@ -225,9 +225,10 @@ Status AggregationNode::get_next(RuntimeState* state, RowBatch* row_batch, bool*
// 1. `!need_finalize` means this aggregation node not the level two aggregation node
// 2. `_singleton_output_tuple != nullptr` means is not group by
// 3. `child(0)->rows_returned() == 0` mean not data from child
// in level two aggregation node should return NULL result
// in level two aggregation node should return nullptr result
// level one aggregation node set `eos = true` return directly
if (UNLIKELY(!_needs_finalize && _singleton_output_tuple != nullptr && child(0)->rows_returned() == 0)) {
if (UNLIKELY(!_needs_finalize && _singleton_output_tuple != nullptr &&
child(0)->rows_returned() == 0)) {
*eos = true;
return Status::OK();
}
Expand Down Expand Up @@ -275,7 +276,7 @@ Status AggregationNode::get_next(RuntimeState* state, RowBatch* row_batch, bool*

*eos = _output_iterator.at_end() || reached_limit();
if (*eos) {
if (_hash_tbl.get() != NULL && _hash_table_buckets_counter != NULL) {
if (_hash_tbl.get() != nullptr && _hash_table_buckets_counter != nullptr) {
COUNTER_SET(_hash_table_buckets_counter, _hash_tbl->num_buckets());
}
}
Expand All @@ -292,8 +293,8 @@ Status AggregationNode::close(RuntimeState* state) {
// them in order to free any memory allocated by UDAs. Finalize() requires a dst tuple
// but we don't actually need the result, so allocate a single dummy tuple to avoid
// accumulating memory.
Tuple* dummy_dst = NULL;
if (_needs_finalize && _output_tuple_desc != NULL) {
Tuple* dummy_dst = nullptr;
if (_needs_finalize && _output_tuple_desc != nullptr) {
dummy_dst = Tuple::create(_output_tuple_desc->byte_size(), _tuple_pool.get());
}
while (!_output_iterator.at_end()) {
Expand All @@ -313,10 +314,10 @@ Status AggregationNode::close(RuntimeState* state) {
}
}

if (_tuple_pool.get() != NULL) {
if (_tuple_pool.get() != nullptr) {
_tuple_pool->free_all();
}
if (_hash_tbl.get() != NULL) {
if (_hash_tbl.get() != nullptr) {
_hash_tbl->close();
}

Expand Down Expand Up @@ -362,17 +363,17 @@ Tuple* AggregationNode::construct_intermediate_tuple() {
// This optimization no longer applies with AnyVal
if (!(*slot_desc)->type().is_string_type() && !(*slot_desc)->type().is_date_type()) {
ExprValue default_value;
void* default_value_ptr = NULL;
void* default_value_ptr = nullptr;

switch (evaluator->agg_op()) {
case TAggregationOp::MIN:
default_value_ptr = default_value.set_to_max((*slot_desc)->type());
RawValue::write(default_value_ptr, agg_tuple, *slot_desc, NULL);
RawValue::write(default_value_ptr, agg_tuple, *slot_desc, nullptr);
break;

case TAggregationOp::MAX:
default_value_ptr = default_value.set_to_min((*slot_desc)->type());
RawValue::write(default_value_ptr, agg_tuple, *slot_desc, NULL);
RawValue::write(default_value_ptr, agg_tuple, *slot_desc, nullptr);
break;

default:
Expand All @@ -385,7 +386,7 @@ Tuple* AggregationNode::construct_intermediate_tuple() {
}

void AggregationNode::update_tuple(Tuple* tuple, TupleRow* row) {
DCHECK(tuple != NULL);
DCHECK(tuple != nullptr);

AggFnEvaluator::add(_aggregate_evaluators, _agg_fn_ctxs, row, tuple);
#if 0
Expand All @@ -404,14 +405,15 @@ void AggregationNode::update_tuple(Tuple* tuple, TupleRow* row) {
}

Tuple* AggregationNode::finalize_tuple(Tuple* tuple, MemPool* pool) {
DCHECK(tuple != NULL);
DCHECK(tuple != nullptr);

Tuple* dst = tuple;
if (_needs_finalize && _intermediate_tuple_id != _output_tuple_id) {
dst = Tuple::create(_output_tuple_desc->byte_size(), pool);
}
if (_needs_finalize) {
AggFnEvaluator::finalize(_aggregate_evaluators, _agg_fn_ctxs, tuple, dst,
AggFnEvaluator::finalize(
_aggregate_evaluators, _agg_fn_ctxs, tuple, dst,
_singleton_output_tuple != nullptr && child(0)->rows_returned() == 0);
} else {
AggFnEvaluator::serialize(_aggregate_evaluators, _agg_fn_ctxs, tuple);
Expand All @@ -424,9 +426,9 @@ Tuple* AggregationNode::finalize_tuple(Tuple* tuple, MemPool* pool) {
SlotDescriptor* src_slot_desc = _intermediate_tuple_desc->slots()[i];
SlotDescriptor* dst_slot_desc = _output_tuple_desc->slots()[i];
bool src_slot_null = tuple->is_null(src_slot_desc->null_indicator_offset());
void* src_slot = NULL;
void* src_slot = nullptr;
if (!src_slot_null) src_slot = tuple->get_slot(src_slot_desc->tuple_offset());
RawValue::write(src_slot, dst, dst_slot_desc, NULL);
RawValue::write(src_slot, dst, dst_slot_desc, nullptr);
}
}
return dst;
Expand Down
7 changes: 3 additions & 4 deletions be/src/exec/aggregation_node.h
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
#ifndef DORIS_BE_SRC_QUERY_EXEC_AGGREGATION_NODE_H
#define DORIS_BE_SRC_QUERY_EXEC_AGGREGATION_NODE_H

#include <boost/scoped_ptr.hpp>
#include <functional>

#include "exec/exec_node.h"
Expand Down Expand Up @@ -64,14 +63,14 @@ class AggregationNode : public ExecNode {
virtual void push_down_predicate(RuntimeState* state, std::list<ExprContext*>* expr_ctxs);

private:
boost::scoped_ptr<HashTable> _hash_tbl;
std::unique_ptr<HashTable> _hash_tbl;
HashTable::Iterator _output_iterator;

std::vector<AggFnEvaluator*> _aggregate_evaluators;

/// FunctionContext for each agg fn and backing pool.
std::vector<doris_udf::FunctionContext*> _agg_fn_ctxs;
boost::scoped_ptr<MemPool> _agg_fn_pool;
std::unique_ptr<MemPool> _agg_fn_pool;

// Exprs used to evaluate input rows
std::vector<ExprContext*> _probe_expr_ctxs;
Expand All @@ -89,7 +88,7 @@ class AggregationNode : public ExecNode {
TupleDescriptor* _output_tuple_desc;

Tuple* _singleton_output_tuple; // result of aggregation w/o GROUP BY
boost::scoped_ptr<MemPool> _tuple_pool;
std::unique_ptr<MemPool> _tuple_pool;

typedef void (*ProcessRowBatchFn)(AggregationNode*, RowBatch*);
// Jitted ProcessRowBatch function pointer. Null if codegen is disabled.
Expand Down
2 changes: 1 addition & 1 deletion be/src/exec/aggregation_node_ir.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ void AggregationNode::process_row_batch_no_grouping(RowBatch* batch, MemPool* po
void AggregationNode::process_row_batch_with_grouping(RowBatch* batch, MemPool* pool) {
for (int i = 0; i < batch->num_rows(); ++i) {
TupleRow* row = batch->get_row(i);
Tuple* agg_tuple = NULL;
Tuple* agg_tuple = nullptr;
HashTable::Iterator it = _hash_tbl->find(row);

if (it.at_end()) {
Expand Down
Loading

0 comments on commit 6c63809

Please sign in to comment.