From 10e1e2971101decaed06c7af830b777765fba25f Mon Sep 17 00:00:00 2001 From: sduzh Date: Thu, 26 Nov 2020 17:00:48 +0800 Subject: [PATCH] Remove header file common/names.h (#4945) --- be/src/agent/agent_server.cpp | 2 +- be/src/agent/cgroups_mgr.cpp | 2 +- be/src/agent/task_worker_pool.cpp | 34 +++--- be/src/agent/utils.cpp | 8 +- be/src/env/env_posix.cpp | 10 +- be/src/env/env_util.cpp | 12 +- be/src/exec/aggregation_node.cpp | 6 +- be/src/exec/analytic_eval_node.cpp | 12 +- be/src/exec/csv_scan_node.cpp | 4 +- be/src/exec/es/es_predicate.cpp | 8 +- be/src/exec/es/es_query_builder.cpp | 4 +- be/src/exec/es/es_scroll_parser.cpp | 10 +- be/src/exec/es_scan_node.cpp | 34 +++--- be/src/exec/exchange_node.cpp | 2 +- be/src/exec/exec_node.cpp | 6 +- be/src/exec/hash_join_node.cpp | 4 +- be/src/exec/hash_table.cpp | 6 +- be/src/exec/merge_join_node.cpp | 4 +- be/src/exec/merge_node.cpp | 8 +- be/src/exec/mysql_scan_node.cpp | 4 +- be/src/exec/odbc_scan_node.cpp | 4 +- be/src/exec/parquet_reader.cpp | 2 +- be/src/exec/partitioned_aggregation_node.cc | 2 +- be/src/exec/partitioned_hash_table.cc | 4 +- be/src/exec/repeat_node.cpp | 6 +- be/src/exec/scan_node.cpp | 8 +- be/src/exec/schema_scan_node.cpp | 4 +- be/src/exec/set_operation_node.cpp | 4 +- be/src/exec/union_node.cpp | 8 +- be/src/exprs/agg_fn.cc | 16 ++- be/src/exprs/agg_fn_evaluator.cpp | 6 +- be/src/exprs/expr.cpp | 14 +-- be/src/exprs/math_functions.cpp | 2 +- be/src/exprs/new_agg_fn_evaluator.cc | 2 +- be/src/exprs/scalar_fn_call.cpp | 2 +- be/src/http/default_path_handlers.cpp | 2 +- be/src/http/web_page_handler.cpp | 6 +- be/src/olap/base_compaction.cpp | 2 +- be/src/olap/compaction.cpp | 6 +- be/src/olap/comparison_predicate.cpp | 26 ++--- be/src/olap/data_dir.cpp | 34 +++--- be/src/olap/delete_handler.cpp | 12 +- be/src/olap/fs/block_id.cpp | 4 +- be/src/olap/fs/file_block_manager.cpp | 12 +- be/src/olap/in_list_predicate.cpp | 24 ++-- be/src/olap/memory/column.cpp | 8 +- be/src/olap/null_predicate.cpp | 2 +- be/src/olap/olap_cond.cpp | 4 +- be/src/olap/olap_server.cpp | 10 +- be/src/olap/olap_snapshot_converter.cpp | 4 +- be/src/olap/options.cpp | 6 +- be/src/olap/push_handler.cpp | 12 +- be/src/olap/row_cursor.cpp | 6 +- be/src/olap/rowset/alpha_rowset.cpp | 4 +- .../rowset/segment_v2/binary_dict_page.cpp | 8 +- .../rowset/segment_v2/binary_prefix_page.cpp | 6 +- .../olap/rowset/segment_v2/column_reader.cpp | 8 +- .../olap/rowset/segment_v2/column_writer.cpp | 4 +- .../segment_v2/indexed_column_reader.cpp | 2 +- be/src/olap/rowset/segment_v2/page_io.cpp | 6 +- be/src/olap/rowset/segment_v2/segment.cpp | 10 +- be/src/olap/schema_change.cpp | 32 ++--- be/src/olap/short_key_index.cpp | 2 +- be/src/olap/snapshot_manager.cpp | 16 +-- be/src/olap/storage_engine.cpp | 22 ++-- be/src/olap/tablet.cpp | 46 ++++---- be/src/olap/tablet_manager.cpp | 20 ++-- be/src/olap/tablet_meta.cpp | 10 +- be/src/olap/task/engine_alter_tablet_task.cpp | 2 +- be/src/olap/task/engine_batch_load_task.cpp | 4 +- be/src/olap/task/engine_clone_task.cpp | 36 +++--- .../olap/task/engine_publish_version_task.cpp | 2 +- .../task/engine_storage_migration_task.cpp | 6 +- be/src/olap/utils.cpp | 6 +- be/src/plugin/plugin_loader.cpp | 2 +- be/src/plugin/plugin_mgr.cpp | 8 +- be/src/plugin/plugin_zip.cpp | 8 +- be/src/runtime/buffered_tuple_stream3.cc | 12 +- be/src/runtime/bufferpool/buffer_allocator.cc | 109 +++++++++--------- be/src/runtime/bufferpool/buffer_allocator.h | 4 +- be/src/runtime/bufferpool/buffer_pool.cc | 56 ++++----- .../runtime/bufferpool/buffer_pool_internal.h | 14 +-- .../runtime/bufferpool/reservation_tracker.cc | 50 ++++---- be/src/runtime/bufferpool/suballocator.cc | 88 +++++++------- be/src/runtime/bufferpool/system_allocator.cc | 2 +- be/src/runtime/initial_reservations.cc | 6 +- be/src/runtime/mem_pool.cpp | 10 +- be/src/runtime/mem_tracker.cpp | 50 ++++---- be/src/runtime/raw_value.cpp | 2 +- be/src/runtime/row_batch.cpp | 24 ++-- be/src/tools/meta_tool.cpp | 12 +- be/src/util/arrow/row_batch.cpp | 4 +- be/src/util/arrow/row_block.cpp | 6 +- be/src/util/block_compression.cpp | 34 +++--- be/src/util/cgroup_util.cpp | 40 +++---- be/src/util/cpu_info.cpp | 58 +++++----- be/src/util/file_utils.cpp | 2 +- be/src/util/os_info.cpp | 13 +-- be/src/util/os_util.cpp | 6 +- be/src/util/path_util.cpp | 20 ++-- be/src/util/perf_counters.cpp | 16 +-- be/src/util/thread.cpp | 4 +- be/src/util/threadpool.cpp | 10 +- be/src/util/time.cpp | 16 +-- be/src/util/trace.cpp | 10 +- be/src/util/zlib.cpp | 4 +- be/test/agent/agent_server_test.cpp | 2 +- be/test/exec/hash_table_test.cpp | 8 +- be/test/exec/new_olap_scan_node_test.cpp | 12 +- be/test/olap/delete_handler_test.cpp | 8 +- be/test/olap/fs/file_block_manager_test.cpp | 8 +- be/test/olap/memory/column_delta_test.cpp | 2 +- be/test/olap/memory/column_test.cpp | 12 +- be/test/olap/memory/mem_tablet_test.cpp | 6 +- .../olap/memory/partial_row_batch_test.cpp | 2 +- be/test/olap/olap_snapshot_converter_test.cpp | 2 +- be/test/olap/rowset/beta_rowset_test.cpp | 2 +- .../segment_v2/column_reader_writer_test.cpp | 30 ++--- .../olap/rowset/segment_v2/segment_test.cpp | 38 +++--- be/test/olap/tablet_mgr_test.cpp | 6 +- be/test/runtime/buffered_block_mgr2_test.cpp | 88 +++++++------- .../runtime/buffered_tuple_stream2_test.cpp | 52 ++++----- be/test/runtime/data_stream_test.cpp | 18 +-- be/test/runtime/disk_io_mgr_test.cpp | 42 +++---- be/test/runtime/memory_scratch_sink_test.cpp | 4 +- be/test/runtime/tmp_file_mgr_test.cpp | 20 ++-- be/test/util/arrow/arrow_work_flow_test.cpp | 4 +- be/test/util/decompress_test.cpp | 4 +- be/test/util/internal_queue_test.cpp | 18 +-- be/test/util/path_util_test.cpp | 2 +- be/test/util/perf_counters_test.cpp | 4 +- be/test/util/rle_encoding_test.cpp | 20 ++-- be/test/util/runtime_profile_test.cpp | 6 +- be/test/util/threadpool_test.cpp | 38 +++--- be/test/util/trace_test.cpp | 6 +- be/test/util/url_coding_test.cpp | 18 +-- 136 files changed, 937 insertions(+), 941 deletions(-) diff --git a/be/src/agent/agent_server.cpp b/be/src/agent/agent_server.cpp index 1f28825866..57ed17a00d 100644 --- a/be/src/agent/agent_server.cpp +++ b/be/src/agent/agent_server.cpp @@ -99,7 +99,7 @@ AgentServer::~AgentServer() { } // TODO(lingbin): each task in the batch may have it own status or FE must check and // resend request when something is wrong(BE may need some logic to guarantee idempotence. -void AgentServer::submit_tasks(TAgentResult& agent_result, const vector& tasks) { +void AgentServer::submit_tasks(TAgentResult& agent_result, const std::vector& tasks) { Status ret_st; // TODO check master_info here if it is the same with that of heartbeat rpc diff --git a/be/src/agent/cgroups_mgr.cpp b/be/src/agent/cgroups_mgr.cpp index 1577428374..128be59397 100644 --- a/be/src/agent/cgroups_mgr.cpp +++ b/be/src/agent/cgroups_mgr.cpp @@ -193,7 +193,7 @@ AgentStatus CgroupsMgr::_config_disk_throttle(std::string user_name, // buld load data path, it is alreay in data path // _exec_env->load_path_mgr()->get_load_data_path(&data_paths); - stringstream ctrl_cmd; + std::stringstream ctrl_cmd; for (auto store : stores) { // check disk type int64_t read_iops = hdd_read_iops; diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp index 575be14cad..fac22f0104 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -325,7 +325,7 @@ void TaskWorkerPool::_create_tablet_worker_thread_callback() { } TStatusCode::type status_code = TStatusCode::OK; - vector error_msgs; + std::vector error_msgs; TStatus task_status; std::vector finish_tablet_infos; @@ -388,7 +388,7 @@ void TaskWorkerPool::_drop_tablet_worker_thread_callback() { } TStatusCode::type status_code = TStatusCode::OK; - vector error_msgs; + std::vector error_msgs; TStatus task_status; TabletSharedPtr dropped_tablet = StorageEngine::instance()->tablet_manager()->get_tablet( drop_tablet_req.tablet_id, drop_tablet_req.schema_hash); @@ -468,7 +468,7 @@ void TaskWorkerPool::_alter_tablet(const TAgentTaskRequest& agent_task_req, int6 TFinishTaskRequest* finish_task_request) { AgentStatus status = DORIS_SUCCESS; TStatus task_status; - vector error_msgs; + std::vector error_msgs; string process_name; switch (task_type) { @@ -516,7 +516,7 @@ void TaskWorkerPool::_alter_tablet(const TAgentTaskRequest& agent_task_req, int6 finish_task_request->__set_task_type(task_type); finish_task_request->__set_signature(signature); - vector finish_tablet_infos; + std::vector finish_tablet_infos; if (status == DORIS_SUCCESS) { TTabletInfo tablet_info; status = _get_tablet_info(new_tablet_id, new_schema_hash, signature, &tablet_info); @@ -602,7 +602,7 @@ void TaskWorkerPool::_push_worker_thread_callback() { LOG(INFO) << "get push task. signature: " << agent_task_req.signature << " priority: " << priority << " push_type: " << push_req.push_type; - vector tablet_infos; + std::vector tablet_infos; EngineBatchLoadTask engine_task(push_req, &tablet_infos, agent_task_req.signature, &status); _env->storage_engine()->execute_task(&engine_task); @@ -613,7 +613,7 @@ void TaskWorkerPool::_push_worker_thread_callback() { continue; } // Return result to fe - vector error_msgs; + std::vector error_msgs; TStatus task_status; TFinishTaskRequest finish_task_request; @@ -676,7 +676,7 @@ void TaskWorkerPool::_publish_version_worker_thread_callback() { LOG(INFO) << "get publish version task, signature:" << agent_task_req.signature; Status st; - vector error_tablet_ids; + std::vector error_tablet_ids; uint32_t retry_time = 0; OLAPStatus res = OLAP_SUCCESS; while (retry_time < PUBLISH_VERSION_MAX_RETRY) { @@ -740,7 +740,7 @@ void TaskWorkerPool::_clear_transaction_task_worker_thread_callback() { << ", partition id size: " << clear_transaction_task_req.partition_id.size(); TStatusCode::type status_code = TStatusCode::OK; - vector error_msgs; + std::vector error_msgs; TStatus task_status; if (clear_transaction_task_req.transaction_id > 0) { @@ -796,7 +796,7 @@ void TaskWorkerPool::_update_tablet_meta_worker_thread_callback() { LOG(INFO) << "get update tablet meta task, signature:" << agent_task_req.signature; TStatusCode::type status_code = TStatusCode::OK; - vector error_msgs; + std::vector error_msgs; TStatus task_status; for (auto tablet_meta_info : update_tablet_meta_req.tabletMetaInfos) { @@ -865,8 +865,8 @@ void TaskWorkerPool::_clone_worker_thread_callback() { DorisMetrics::instance()->clone_requests_total->increment(1); LOG(INFO) << "get clone task. signature:" << agent_task_req.signature; - vector error_msgs; - vector tablet_infos; + std::vector error_msgs; + std::vector tablet_infos; EngineCloneTask engine_task(clone_req, _master_info, agent_task_req.signature, &error_msgs, &tablet_infos, &status); _env->storage_engine()->execute_task(&engine_task); @@ -935,7 +935,7 @@ void TaskWorkerPool::_storage_medium_migrate_worker_thread_callback() { } TStatus task_status; - vector error_msgs; + std::vector error_msgs; task_status.__set_status_code(status_code); task_status.__set_error_msgs(error_msgs); @@ -1030,7 +1030,7 @@ void TaskWorkerPool::_check_consistency_worker_thread_callback() { } TStatusCode::type status_code = TStatusCode::OK; - vector error_msgs; + std::vector error_msgs; TStatus task_status; uint32_t checksum = 0; @@ -1113,7 +1113,7 @@ void TaskWorkerPool::_report_disk_state_worker_thread_callback() { break; } - vector data_dir_infos; + std::vector data_dir_infos; _env->storage_engine()->get_all_data_dir_info(&data_dir_infos, true /* update */); map disks; @@ -1332,7 +1332,7 @@ void TaskWorkerPool::_make_snapshot_thread_callback() { LOG(INFO) << "get snapshot task, signature:" << agent_task_req.signature; TStatusCode::type status_code = TStatusCode::OK; - vector error_msgs; + std::vector error_msgs; TStatus task_status; string snapshot_path; @@ -1413,7 +1413,7 @@ void TaskWorkerPool::_release_snapshot_thread_callback() { LOG(INFO) << "get release snapshot task, signature:" << agent_task_req.signature; TStatusCode::type status_code = TStatusCode::OK; - vector error_msgs; + std::vector error_msgs; TStatus task_status; string& snapshot_path = release_snapshot_request.snapshot_path; @@ -1482,7 +1482,7 @@ void TaskWorkerPool::_move_dir_thread_callback() { << ", job id:" << move_dir_req.job_id; TStatusCode::type status_code = TStatusCode::OK; - vector error_msgs; + std::vector error_msgs; TStatus task_status; // TODO: move dir diff --git a/be/src/agent/utils.cpp b/be/src/agent/utils.cpp index 0a419f613e..a24effadd9 100644 --- a/be/src/agent/utils.cpp +++ b/be/src/agent/utils.cpp @@ -147,11 +147,11 @@ AgentStatus AgentUtils::rsync_from_remote( const string& remote_host, const string& remote_file_path, const string& local_file_path, - const vector& exclude_file_patterns, + const std::vector& exclude_file_patterns, uint32_t transport_speed_limit_kbps, uint32_t timeout_second) { int ret_code = 0; - stringstream cmd_stream; + std::stringstream cmd_stream; cmd_stream << "rsync -r -q -e \"ssh -o StrictHostKeyChecking=no\""; for (auto exclude_file_pattern : exclude_file_patterns) { cmd_stream << " --exclude=" << exclude_file_pattern; @@ -240,7 +240,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) { - stringstream err_stream; + std::stringstream err_stream; err_stream << "popen failed. " << strerror(errno) << ", with errno: " << errno << ".\n"; *errmsg = err_stream.str(); return false; @@ -258,7 +258,7 @@ bool AgentUtils::exec_cmd(const string& command, string* errmsg, bool redirect_s if (errno == ECHILD) { *errmsg += "pclose cannot obtain the child status.\n"; } else { - stringstream err_stream; + std::stringstream err_stream; err_stream << "Close popen failed. " << strerror(errno) << ", with errno: " << errno << "\n"; *errmsg += err_stream.str(); diff --git a/be/src/env/env_posix.cpp b/be/src/env/env_posix.cpp index bfd7fb4d2c..dd9bdc2bcd 100644 --- a/be/src/env/env_posix.cpp +++ b/be/src/env/env_posix.cpp @@ -90,7 +90,7 @@ static Status do_open(const string& filename, Env::OpenMode mode, int* fd) { case Env::MUST_EXIST: break; default: - return Status::NotSupported(Substitute("Unknown create mode $0", mode)); + return Status::NotSupported(strings::Substitute("Unknown create mode $0", mode)); } int f; RETRY_ON_EINTR(f, open(filename.c_str(), flags, 0666)); @@ -128,7 +128,7 @@ static Status do_readv_at(int fd, const std::string& filename, uint64_t offset, if (PREDICT_FALSE(r == 0)) { return Status::EndOfFile( - Substitute("EOF trying to read $0 bytes at offset $1", bytes_req, offset)); + strings::Substitute("EOF trying to read $0 bytes at offset $1", bytes_req, offset)); } if (PREDICT_TRUE(r == rem)) { @@ -252,7 +252,7 @@ public: const string& filename() const override { return _filename; } private: - const string _filename; + const std::string _filename; FILE* const _file; }; @@ -494,7 +494,7 @@ public: } private: - const string _filename; + const std::string _filename; const int _fd; const bool _sync_on_close = false; bool _closed = false; @@ -675,7 +675,7 @@ public: // because the buffer is allocated by malloc(), see `man 3 realpath`. std::unique_ptr r(realpath(path.c_str(), nullptr)); if (r == nullptr) { - return io_error(Substitute("Unable to canonicalize $0", path), errno); + return io_error(strings::Substitute("Unable to canonicalize $0", path), errno); } *result = std::string(r.get()); return Status::OK(); diff --git a/be/src/env/env_util.cpp b/be/src/env/env_util.cpp index b383439b76..2c994e94c3 100644 --- a/be/src/env/env_util.cpp +++ b/be/src/env/env_util.cpp @@ -22,7 +22,7 @@ using std::shared_ptr; using std::string; -using std::unique_ptr; + namespace doris { namespace env_util { @@ -33,14 +33,14 @@ Status open_file_for_write(Env* env, const string& path, shared_ptr* file) { - unique_ptr w; + std::unique_ptr w; RETURN_IF_ERROR(env->new_writable_file(opts, path, &w)); file->reset(w.release()); return Status::OK(); } Status open_file_for_random(Env* env, const string& path, shared_ptr* file) { - unique_ptr r; + std::unique_ptr r; RETURN_IF_ERROR(env->new_random_access_file(path, &r)); file->reset(r.release()); return Status::OK(); @@ -48,7 +48,7 @@ Status open_file_for_random(Env* env, const string& path, shared_ptr file; + std::unique_ptr file; Status s = env->new_writable_file(fname, &file); if (!s.ok()) { return s; @@ -78,13 +78,13 @@ Status write_string_to_file_sync(Env* env, const Slice& data, const std::string& Status read_file_to_string(Env* env, const std::string& fname, faststring* data) { data->clear(); - unique_ptr file; + std::unique_ptr file; Status s = env->new_sequential_file(fname, &file); if (!s.ok()) { return s; } static const int kBufferSize = 8192; - unique_ptr scratch(new uint8_t[kBufferSize]); + std::unique_ptr scratch(new uint8_t[kBufferSize]); while (true) { Slice fragment(scratch.get(), kBufferSize); s = file->read(&fragment); diff --git a/be/src/exec/aggregation_node.cpp b/be/src/exec/aggregation_node.cpp index 4768f32655..53e33268f9 100644 --- a/be/src/exec/aggregation_node.cpp +++ b/be/src/exec/aggregation_node.cpp @@ -136,7 +136,7 @@ Status AggregationNode::prepare(RuntimeState* state) { // TODO: how many buckets? _hash_tbl.reset(new HashTable( _build_expr_ctxs, _probe_expr_ctxs, 1, true, - vector(_build_expr_ctxs.size(), false), id(), mem_tracker(), 1024)); + std::vector(_build_expr_ctxs.size(), false), id(), mem_tracker(), 1024)); if (_probe_expr_ctxs.empty()) { // create single output tuple now; we need to output something @@ -328,7 +328,7 @@ Status AggregationNode::close(RuntimeState* state) { Tuple* AggregationNode::construct_intermediate_tuple() { Tuple* agg_tuple = Tuple::create(_intermediate_tuple_desc->byte_size(), _tuple_pool.get()); - vector::const_iterator slot_desc = _intermediate_tuple_desc->slots().begin(); + std::vector::const_iterator slot_desc = _intermediate_tuple_desc->slots().begin(); // copy grouping values for (int i = 0; i < _probe_expr_ctxs.size(); ++i, ++slot_desc) { @@ -389,7 +389,7 @@ void AggregationNode::update_tuple(Tuple* tuple, TupleRow* row) { AggFnEvaluator::add(_aggregate_evaluators, _agg_fn_ctxs, row, tuple); #if 0 - vector::const_iterator evaluator; + std::vector::const_iterator evaluator; int i = 0; for (evaluator = _aggregate_evaluators.begin(); evaluator != _aggregate_evaluators.end(); ++evaluator, ++i) { diff --git a/be/src/exec/analytic_eval_node.cpp b/be/src/exec/analytic_eval_node.cpp index 099affbb0f..6aae452777 100644 --- a/be/src/exec/analytic_eval_node.cpp +++ b/be/src/exec/analytic_eval_node.cpp @@ -164,10 +164,10 @@ Status AnalyticEvalNode::prepare(RuntimeState* state) { if (_partition_by_eq_expr_ctx != NULL || _order_by_eq_expr_ctx != NULL) { DCHECK(_buffered_tuple_desc != NULL); - vector tuple_ids; + std::vector tuple_ids; tuple_ids.push_back(child(0)->row_desc().tuple_descriptors()[0]->id()); tuple_ids.push_back(_buffered_tuple_desc->id()); - RowDescriptor cmp_row_desc(state->desc_tbl(), tuple_ids, vector(2, false)); + RowDescriptor cmp_row_desc(state->desc_tbl(), tuple_ids, std::vector(2, false)); if (_partition_by_eq_expr_ctx != NULL) { RETURN_IF_ERROR( @@ -261,12 +261,12 @@ Status AnalyticEvalNode::open(RuntimeState* state) { return Status::OK(); } -string debug_window_bound_string(const TAnalyticWindowBoundary& b) { +std::string debug_window_bound_string(const TAnalyticWindowBoundary& b) { if (b.type == TAnalyticWindowBoundaryType::CURRENT_ROW) { return "CURRENT_ROW"; } - stringstream ss; + std::stringstream ss; if (b.__isset.rows_offset_value) { ss << b.rows_offset_value; @@ -321,7 +321,7 @@ std::string AnalyticEvalNode::debug_window_string() const { } std::string AnalyticEvalNode::debug_state_string(bool detailed) const { - stringstream ss; + std::stringstream ss; ss << "num_returned=" << _input_stream->rows_returned() << " num_rows=" << _input_stream->num_rows() << " _curr_partition_idx=" << _curr_partition_idx @@ -910,7 +910,7 @@ Status AnalyticEvalNode::close(RuntimeState* state) { return Status::OK(); } -void AnalyticEvalNode::debug_string(int indentation_level, stringstream* out) const { +void AnalyticEvalNode::debug_string(int indentation_level, std::stringstream* out) const { *out << string(indentation_level * 2, ' '); *out << "AnalyticEvalNode(" << " window=" << debug_window_string(); diff --git a/be/src/exec/csv_scan_node.cpp b/be/src/exec/csv_scan_node.cpp index a1ef55a8bd..58baecc43f 100644 --- a/be/src/exec/csv_scan_node.cpp +++ b/be/src/exec/csv_scan_node.cpp @@ -361,7 +361,7 @@ Status CsvScanNode::close(RuntimeState* state) { return Status::OK(); } -void CsvScanNode::debug_string(int indentation_level, stringstream* out) const { +void CsvScanNode::debug_string(int indentation_level, std::stringstream* out) const { *out << string(indentation_level * 2, ' '); *out << "csvScanNode(tupleid=" << _tuple_id; *out << ")" << std::endl; @@ -371,7 +371,7 @@ void CsvScanNode::debug_string(int indentation_level, stringstream* out) const { } } -Status CsvScanNode::set_scan_ranges(const vector& scan_ranges) { +Status CsvScanNode::set_scan_ranges(const std::vector& scan_ranges) { return Status::OK(); } diff --git a/be/src/exec/es/es_predicate.cpp b/be/src/exec/es/es_predicate.cpp index 6c6b609c99..1070b45f9b 100644 --- a/be/src/exec/es/es_predicate.cpp +++ b/be/src/exec/es/es_predicate.cpp @@ -196,7 +196,7 @@ Status EsPredicate::build_disjuncts_list() { } // make sure to build by build_disjuncts_list -const vector& EsPredicate::get_predicate_list(){ +const std::vector& EsPredicate::get_predicate_list(){ return _disjuncts; } @@ -292,9 +292,9 @@ Status EsPredicate::build_disjuncts_list(const Expr* conjunct) { } Expr* expr = conjunct->get_child(1); ExtLiteral literal(expr->type().type, _context->get_value(expr, NULL)); - vector query_conditions; + std::vector query_conditions; query_conditions.emplace_back(literal); - vector cols; + std::vector cols; ExtPredicate* predicate = new ExtFunction( TExprNodeType::FUNCTION_CALL, "esquery", @@ -383,7 +383,7 @@ Status EsPredicate::build_disjuncts_list(const Expr* conjunct) { "opcode in IN_PRED is neither FILTER_IN nor FILTER_NOT_IN"); } - vector in_pred_values; + std::vector in_pred_values; const InPredicate* pred = dynamic_cast(conjunct); const Expr* expr = Expr::expr_without_cast(pred->get_child(0)); if (expr->node_type() != TExprNodeType::SLOT_REF) { diff --git a/be/src/exec/es/es_query_builder.cpp b/be/src/exec/es/es_query_builder.cpp index a448ecc7c0..441aa55156 100644 --- a/be/src/exec/es/es_query_builder.cpp +++ b/be/src/exec/es/es_query_builder.cpp @@ -268,7 +268,7 @@ BooleanQueryBuilder::BooleanQueryBuilder(const std::vector& predi if (compound_predicates->op == TExprOpcode::COMPOUND_AND) { BooleanQueryBuilder* bool_query = new BooleanQueryBuilder(); for (auto es_predicate : compound_predicates->conjuncts) { - vector or_predicates = es_predicate->get_predicate_list(); + std::vector or_predicates = es_predicate->get_predicate_list(); BooleanQueryBuilder* inner_bool_query = new BooleanQueryBuilder(or_predicates); bool_query->must(inner_bool_query); } @@ -434,7 +434,7 @@ void BooleanQueryBuilder::to_query(const std::vector& predicates, root->SetObject(); BooleanQueryBuilder bool_query; for (auto es_predicate : predicates) { - vector or_predicates = es_predicate->get_predicate_list(); + std::vector or_predicates = es_predicate->get_predicate_list(); BooleanQueryBuilder* inner_bool_query = new BooleanQueryBuilder(or_predicates); bool_query.must(inner_bool_query); } diff --git a/be/src/exec/es/es_scroll_parser.cpp b/be/src/exec/es/es_scroll_parser.cpp index 4e0ccfe517..4bf44dc5e5 100644 --- a/be/src/exec/es/es_scroll_parser.cpp +++ b/be/src/exec/es/es_scroll_parser.cpp @@ -71,12 +71,12 @@ std::string json_value_to_string(const rapidjson::Value& value) { return scratch_buffer.GetString(); } -static const string ERROR_INVALID_COL_DATA = "Data source returned inconsistent column data. " +static const std::string ERROR_INVALID_COL_DATA = "Data source returned inconsistent column data. " "Expected value of type $0 based on column metadata. This likely indicates a " "problem with the data source library."; -static const string ERROR_MEM_LIMIT_EXCEEDED = "DataSourceScanNode::$0() failed to allocate " +static const std::string ERROR_MEM_LIMIT_EXCEEDED = "DataSourceScanNode::$0() failed to allocate " "$1 bytes for $2."; -static const string ERROR_COL_DATA_IS_ARRAY = "Data source returned an array for the type $0" +static const std::string ERROR_COL_DATA_IS_ARRAY = "Data source returned an array for the type $0" "based on column metadata."; #define RETURN_ERROR_IF_COL_IS_ARRAY(col, type) \ @@ -301,7 +301,7 @@ Status ScrollParser::fill_tuple(const TupleDescriptor* tuple_desc, size_t len = _id.length(); char* buffer = reinterpret_cast(tuple_pool->try_allocate_unaligned(len)); if (UNLIKELY(buffer == NULL)) { - string details = strings::Substitute(ERROR_MEM_LIMIT_EXCEEDED, "MaterializeNextRow", + std::string details = strings::Substitute(ERROR_MEM_LIMIT_EXCEEDED, "MaterializeNextRow", len, "string slot"); return tuple_pool->mem_tracker()->MemLimitExceeded(NULL, details, len); } @@ -351,7 +351,7 @@ Status ScrollParser::fill_tuple(const TupleDescriptor* tuple_desc, size_t val_size = val.length(); char* buffer = reinterpret_cast(tuple_pool->try_allocate_unaligned(val_size)); if (UNLIKELY(buffer == NULL)) { - string details = strings::Substitute(ERROR_MEM_LIMIT_EXCEEDED, "MaterializeNextRow", + std::string details = strings::Substitute(ERROR_MEM_LIMIT_EXCEEDED, "MaterializeNextRow", val_size, "string slot"); return tuple_pool->mem_tracker()->MemLimitExceeded(NULL, details, val_size); } diff --git a/be/src/exec/es_scan_node.cpp b/be/src/exec/es_scan_node.cpp index 54ae643625..abde624931 100644 --- a/be/src/exec/es_scan_node.cpp +++ b/be/src/exec/es_scan_node.cpp @@ -41,10 +41,10 @@ namespace doris { // $0 = column type (e.g. INT) -const string ERROR_INVALID_COL_DATA = "Data source returned inconsistent column data. " +const std::string ERROR_INVALID_COL_DATA = "Data source returned inconsistent column data. " "Expected value of type $0 based on column metadata. This likely indicates a " "problem with the data source library."; -const string ERROR_MEM_LIMIT_EXCEEDED = "DataSourceScanNode::$0() failed to allocate " +const std::string ERROR_MEM_LIMIT_EXCEEDED = "DataSourceScanNode::$0() failed to allocate " "$1 bytes for $2."; EsScanNode::EsScanNode( @@ -87,7 +87,7 @@ Status EsScanNode::open(RuntimeState* state) { RETURN_IF_ERROR(ExecNode::open(state)); // TExtOpenParams.row_schema - vector cols; + std::vector cols; for (const SlotDescriptor* slot : _tuple_desc->slots()) { TExtColumnDesc col; col.__set_name(slot->col_name()); @@ -99,11 +99,11 @@ Status EsScanNode::open(RuntimeState* state) { row_schema.__isset.cols = true; // TExtOpenParams.predicates - vector> predicates; - vector predicate_to_conjunct; + std::vector> predicates; + std::vector predicate_to_conjunct; for (int i = 0; i < _conjunct_ctxs.size(); ++i) { VLOG(1) << "conjunct: " << _conjunct_ctxs[i]->root()->debug_string(); - vector disjuncts; + std::vector disjuncts; if (get_disjuncts(_conjunct_ctxs[i], _conjunct_ctxs[i]->root(), disjuncts)) { predicates.emplace_back(std::move(disjuncts)); predicate_to_conjunct.push_back(i); @@ -111,7 +111,7 @@ Status EsScanNode::open(RuntimeState* state) { } // open every scan range - vector conjunct_accepted_times(_conjunct_ctxs.size(), 0); + std::vector conjunct_accepted_times(_conjunct_ctxs.size(), 0); for (int i = 0; i < _scan_ranges.size(); ++i) { TEsScanRange& es_scan_range = _scan_ranges[i]; @@ -222,9 +222,9 @@ Status EsScanNode::get_next(RuntimeState* state, RowBatch* row_batch, bool* eos) // convert VLOG(1) << "begin to convert: scan_range_idx=" << _scan_range_idx << ", num_rows=" << result.rows.num_rows; - vector& cols = result.rows.cols; + std::vector& cols = result.rows.cols; // indexes of the next non-null value in the row batch, per column. - vector cols_next_val_idx(_tuple_desc->slots().size(), 0); + std::vector cols_next_val_idx(_tuple_desc->slots().size(), 0); for (int row_idx = 0; row_idx < result.rows.num_rows; row_idx++) { if (reached_limit()) { *eos = true; @@ -311,7 +311,7 @@ Status EsScanNode::close(RuntimeState* state) { return Status::OK(); } -void EsScanNode::debug_string(int indentation_level, stringstream* out) const { +void EsScanNode::debug_string(int indentation_level, std::stringstream* out) const { *out << string(indentation_level * 2, ' '); *out << "EsScanNode(tupleid=" << _tuple_id; *out << ")" << std::endl; @@ -321,7 +321,7 @@ void EsScanNode::debug_string(int indentation_level, stringstream* out) const { } } -Status EsScanNode::set_scan_ranges(const vector& scan_ranges) { +Status EsScanNode::set_scan_ranges(const std::vector& scan_ranges) { for (int i = 0; i < scan_ranges.size(); ++i) { TScanRangeParams scan_range = scan_ranges[i]; DCHECK(scan_range.scan_range.__isset.es_scan_range); @@ -396,7 +396,7 @@ bool EsScanNode::ignore_cast(SlotDescriptor* slot, Expr* expr) { } bool EsScanNode::get_disjuncts(ExprContext* context, Expr* conjunct, - vector& disjuncts) { + std::vector& disjuncts) { if (TExprNodeType::BINARY_PRED == conjunct->node_type()) { if (conjunct->children().size() != 2) { VLOG(1) << "get disjuncts fail: number of children is not 2"; @@ -448,7 +448,7 @@ bool EsScanNode::get_disjuncts(ExprContext* context, Expr* conjunct, // down it to es TExtFunction match_function; match_function.__set_func_name(conjunct->fn().name.function_name); - vector query_conditions; + std::vector query_conditions; TExtLiteral literal; @@ -473,7 +473,7 @@ bool EsScanNode::get_disjuncts(ExprContext* context, Expr* conjunct, return false; } TExtInPredicate ext_in_predicate; - vector in_pred_values; + std::vector in_pred_values; InPredicate* pred = dynamic_cast(conjunct); ext_in_predicate.__set_is_not_in(pred->is_not_in()); if (Expr::type_without_cast(pred->get_child(0)) != TExprNodeType::SLOT_REF) { @@ -754,8 +754,8 @@ Status EsScanNode::get_next_from_es(TExtGetNextResult& result) { } Status EsScanNode::materialize_row(MemPool* tuple_pool, Tuple* tuple, - const vector& cols, int row_idx, - vector& cols_next_val_idx) { + const std::vector& cols, int row_idx, + std::vector& cols_next_val_idx) { tuple->init(_tuple_desc->byte_size()); for (int i = 0; i < _tuple_desc->slots().size(); ++i) { @@ -786,7 +786,7 @@ Status EsScanNode::materialize_row(MemPool* tuple_pool, Tuple* tuple, size_t val_size = val.size(); char* buffer = reinterpret_cast(tuple_pool->try_allocate_unaligned(val_size)); if (UNLIKELY(buffer == NULL)) { - string details = strings::Substitute(ERROR_MEM_LIMIT_EXCEEDED, "MaterializeNextRow", + std::string details = strings::Substitute(ERROR_MEM_LIMIT_EXCEEDED, "MaterializeNextRow", val_size, "string slot"); return tuple_pool->mem_tracker()->MemLimitExceeded(NULL, details, val_size); } diff --git a/be/src/exec/exchange_node.cpp b/be/src/exec/exchange_node.cpp index 6c41f77d86..39a688ab80 100644 --- a/be/src/exec/exchange_node.cpp +++ b/be/src/exec/exchange_node.cpp @@ -37,7 +37,7 @@ ExchangeNode::ExchangeNode( _num_senders(0), _stream_recvr(NULL), _input_row_desc(descs, tnode.exchange_node.input_row_tuples, - vector( + std::vector( tnode.nullable_tuples.begin(), tnode.nullable_tuples.begin() + tnode.exchange_node.input_row_tuples.size())), _next_row_idx(0), diff --git a/be/src/exec/exec_node.cpp b/be/src/exec/exec_node.cpp index 4713864ded..1f9531343a 100644 --- a/be/src/exec/exec_node.cpp +++ b/be/src/exec/exec_node.cpp @@ -285,7 +285,7 @@ Status ExecNode::create_tree(RuntimeState* state, ObjectPool* pool, const TPlan& Status ExecNode::create_tree_helper( RuntimeState* state, ObjectPool* pool, - const vector& tnodes, + const std::vector& tnodes, const DescriptorTbl& descs, ExecNode* parent, int* node_idx, @@ -509,7 +509,7 @@ bool ExecNode::eval_conjuncts(ExprContext* const* ctxs, int num_ctxs, TupleRow* return true; } -void ExecNode::collect_nodes(TPlanNodeType::type node_type, vector* nodes) { +void ExecNode::collect_nodes(TPlanNodeType::type node_type, std::vector* nodes) { if (_type == node_type) { nodes->push_back(this); } @@ -617,7 +617,7 @@ Status ExecNode::enable_deny_reservation_debug_action() { debug_action_param_.c_str(), debug_action_param_.size(), &parse_result); if (parse_result != StringParser::PARSE_SUCCESS || probability < 0.0 || probability > 1.0) { - return Status::InternalError(Substitute( + return Status::InternalError(strings::Substitute( "Invalid SET_DENY_RESERVATION_PROBABILITY param: '$0'", debug_action_param_)); } _buffer_pool_client.SetDebugDenyIncreaseReservation(probability); diff --git a/be/src/exec/hash_join_node.cpp b/be/src/exec/hash_join_node.cpp index c120330daf..4b55462caa 100644 --- a/be/src/exec/hash_join_node.cpp +++ b/be/src/exec/hash_join_node.cpp @@ -55,7 +55,7 @@ HashJoinNode::~HashJoinNode() { Status HashJoinNode::init(const TPlanNode& tnode, RuntimeState* state) { RETURN_IF_ERROR(ExecNode::init(tnode, state)); DCHECK(tnode.__isset.hash_join_node); - const vector& eq_join_conjuncts = tnode.hash_join_node.eq_join_conjuncts; + const std::vector& eq_join_conjuncts = tnode.hash_join_node.eq_join_conjuncts; for (int i = 0; i < eq_join_conjuncts.size(); ++i) { ExprContext* ctx = NULL; @@ -681,7 +681,7 @@ Status HashJoinNode::left_join_get_next(RuntimeState* state, return Status::OK(); } -string HashJoinNode::get_probe_row_output_string(TupleRow* probe_row) { +std::string HashJoinNode::get_probe_row_output_string(TupleRow* probe_row) { std::stringstream out; out << "["; int* _build_tuple_idx_ptr = &_build_tuple_idx[0]; diff --git a/be/src/exec/hash_table.cpp b/be/src/exec/hash_table.cpp index 48b70b52f8..471663dc0a 100644 --- a/be/src/exec/hash_table.cpp +++ b/be/src/exec/hash_table.cpp @@ -28,8 +28,8 @@ namespace doris { const float HashTable::MAX_BUCKET_OCCUPANCY_FRACTION = 0.75f; -HashTable::HashTable(const vector& build_expr_ctxs, - const vector& probe_expr_ctxs, +HashTable::HashTable(const std::vector& build_expr_ctxs, + const std::vector& probe_expr_ctxs, int num_build_tuples, bool stores_nulls, const std::vector& finds_nulls, int32_t initial_seed, @@ -85,7 +85,7 @@ void HashTable::close() { _mem_tracker->Release(_buckets.size() * sizeof(Bucket)); } -bool HashTable::eval_row(TupleRow* row, const vector& ctxs) { +bool HashTable::eval_row(TupleRow* row, const std::vector& ctxs) { // Put a non-zero constant in the result location for NULL. // We don't want(NULL, 1) to hash to the same as (0, 1). // This needs to be as big as the biggest primitive type since the bytes diff --git a/be/src/exec/merge_join_node.cpp b/be/src/exec/merge_join_node.cpp index 9121940dc6..12a257199b 100644 --- a/be/src/exec/merge_join_node.cpp +++ b/be/src/exec/merge_join_node.cpp @@ -57,7 +57,7 @@ MergeJoinNode::~MergeJoinNode() { Status MergeJoinNode::init(const TPlanNode& tnode, RuntimeState* state) { DCHECK(tnode.__isset.merge_join_node); RETURN_IF_ERROR(ExecNode::init(tnode, state)); - const vector& cmp_conjuncts = + const std::vector& cmp_conjuncts = tnode.merge_join_node.cmp_conjuncts; for (int i = 0; i < cmp_conjuncts.size(); ++i) { @@ -326,7 +326,7 @@ Status MergeJoinNode::get_input_row(RuntimeState* state, int child_idx) { return Status::OK(); } -void MergeJoinNode::debug_string(int indentation_level, stringstream* out) const { +void MergeJoinNode::debug_string(int indentation_level, std::stringstream* out) const { *out << string(indentation_level * 2, ' '); *out << "MergeJoin(eos=" << (_eos ? "true" : "false") << " _left_child_pos=" << (_left_child_ctx.get() ? _left_child_ctx->row_idx : -1) diff --git a/be/src/exec/merge_node.cpp b/be/src/exec/merge_node.cpp index a979d95d54..92152ecb72 100644 --- a/be/src/exec/merge_node.cpp +++ b/be/src/exec/merge_node.cpp @@ -42,16 +42,16 @@ Status MergeNode::init(const TPlanNode& tnode, RuntimeState* state) { RETURN_IF_ERROR(ExecNode::init(tnode, state)); DCHECK(tnode.__isset.merge_node); // Create _const_expr_lists from thrift exprs. - const vector>& const_texpr_lists = tnode.merge_node.const_expr_lists; + const std::vector>& const_texpr_lists = tnode.merge_node.const_expr_lists; for (int i = 0; i < const_texpr_lists.size(); ++i) { - vector ctxs; + std::vector ctxs; RETURN_IF_ERROR(Expr::create_expr_trees(_pool, const_texpr_lists[i], &ctxs)); _const_result_expr_ctx_lists.push_back(ctxs); } // Create _result_expr__ctx_lists from thrift exprs. - const vector>& result_texpr_lists = tnode.merge_node.result_expr_lists; + const std::vector>& result_texpr_lists = tnode.merge_node.result_expr_lists; for (int i = 0; i < result_texpr_lists.size(); ++i) { - vector ctxs; + std::vector ctxs; RETURN_IF_ERROR(Expr::create_expr_trees(_pool, result_texpr_lists[i], &ctxs)); _result_expr_ctx_lists.push_back(ctxs); } diff --git a/be/src/exec/mysql_scan_node.cpp b/be/src/exec/mysql_scan_node.cpp index f76fcfee9e..7f087f3971 100644 --- a/be/src/exec/mysql_scan_node.cpp +++ b/be/src/exec/mysql_scan_node.cpp @@ -250,7 +250,7 @@ Status MysqlScanNode::close(RuntimeState* state) { return ExecNode::close(state); } -void MysqlScanNode::debug_string(int indentation_level, stringstream* out) const { +void MysqlScanNode::debug_string(int indentation_level, std::stringstream* out) const { *out << string(indentation_level * 2, ' '); *out << "MysqlScanNode(tupleid=" << _tuple_id << " table=" << _table_name; *out << ")" << std::endl; @@ -260,7 +260,7 @@ void MysqlScanNode::debug_string(int indentation_level, stringstream* out) const } } -Status MysqlScanNode::set_scan_ranges(const vector& scan_ranges) { +Status MysqlScanNode::set_scan_ranges(const std::vector& scan_ranges) { return Status::OK(); } diff --git a/be/src/exec/odbc_scan_node.cpp b/be/src/exec/odbc_scan_node.cpp index 55614c5924..91deca2854 100644 --- a/be/src/exec/odbc_scan_node.cpp +++ b/be/src/exec/odbc_scan_node.cpp @@ -239,7 +239,7 @@ Status OdbcScanNode::close(RuntimeState* state) { return ExecNode::close(state); } -void OdbcScanNode::debug_string(int indentation_level, stringstream* out) const { +void OdbcScanNode::debug_string(int indentation_level, std::stringstream* out) const { *out << string(indentation_level * 2, ' '); *out << "OdbcScanNode(tupleid=" << _tuple_id << " table=" << _table_name; *out << ")" << std::endl; @@ -249,7 +249,7 @@ void OdbcScanNode::debug_string(int indentation_level, stringstream* out) const } } -Status OdbcScanNode::set_scan_ranges(const vector& scan_ranges) { +Status OdbcScanNode::set_scan_ranges(const std::vector& scan_ranges) { return Status::OK(); } diff --git a/be/src/exec/parquet_reader.cpp b/be/src/exec/parquet_reader.cpp index dd4bfe9fa9..6ed2020650 100644 --- a/be/src/exec/parquet_reader.cpp +++ b/be/src/exec/parquet_reader.cpp @@ -310,7 +310,7 @@ Status ParquetReaderWrap::read(Tuple* tuple, const std::vector& if (fixed_array->IsNull(_current_line_of_batch)) { RETURN_IF_ERROR(set_field_null(tuple, slot_desc)); } else { - string value = fixed_array->GetString(_current_line_of_batch); + std::string value = fixed_array->GetString(_current_line_of_batch); fill_slot(tuple, slot_desc, mem_pool, (uint8_t*)value.c_str(), value.length()); } break; diff --git a/be/src/exec/partitioned_aggregation_node.cc b/be/src/exec/partitioned_aggregation_node.cc index 2c767e2afe..feab536c8c 100644 --- a/be/src/exec/partitioned_aggregation_node.cc +++ b/be/src/exec/partitioned_aggregation_node.cc @@ -46,7 +46,7 @@ #include "gen_cpp/Exprs_types.h" #include "gen_cpp/PlanNodes_types.h" -#include "common/names.h" + using namespace strings; diff --git a/be/src/exec/partitioned_hash_table.cc b/be/src/exec/partitioned_hash_table.cc index e8ab74a2f8..de79e1da30 100644 --- a/be/src/exec/partitioned_hash_table.cc +++ b/be/src/exec/partitioned_hash_table.cc @@ -32,7 +32,7 @@ #include "runtime/string_value.h" #include "util/doris_metrics.h" -#include "common/names.h" + using namespace doris; using namespace strings; @@ -155,7 +155,7 @@ Status PartitionedHashTableCtx::Create(ObjectPool* pool, RuntimeState* state, int num_build_tuples, MemPool* mem_pool, MemPool* expr_results_pool, const std::shared_ptr& tracker, const RowDescriptor& row_desc, const RowDescriptor& row_desc_probe, - scoped_ptr* ht_ctx) { + boost::scoped_ptr* ht_ctx) { ht_ctx->reset(new PartitionedHashTableCtx(build_exprs, probe_exprs, stores_nulls, finds_nulls, initial_seed, max_levels, mem_pool, expr_results_pool, tracker)); return (*ht_ctx)->Init(pool, state, num_build_tuples, row_desc, row_desc_probe); diff --git a/be/src/exec/repeat_node.cpp b/be/src/exec/repeat_node.cpp index bfc22f5793..7997770d72 100644 --- a/be/src/exec/repeat_node.cpp +++ b/be/src/exec/repeat_node.cpp @@ -78,9 +78,9 @@ Status RepeatNode::get_repeated_batch( // Fill all slots according to child MemPool* tuple_pool = row_batch->tuple_data_pool(); - const vector& src_tuple_descs = child_row_batch->row_desc().tuple_descriptors(); - const vector& dst_tuple_descs = row_batch->row_desc().tuple_descriptors(); - vector dst_tuples(src_tuple_descs.size(), nullptr); + const std::vector& src_tuple_descs = child_row_batch->row_desc().tuple_descriptors(); + const std::vector& dst_tuple_descs = row_batch->row_desc().tuple_descriptors(); + std::vector dst_tuples(src_tuple_descs.size(), nullptr); for (int i = 0; i < child_row_batch->num_rows(); ++i) { int row_idx = row_batch->add_row(); TupleRow* dst_row = row_batch->get_row(row_idx); diff --git a/be/src/exec/scan_node.cpp b/be/src/exec/scan_node.cpp index 5654f052c0..f89b206611 100644 --- a/be/src/exec/scan_node.cpp +++ b/be/src/exec/scan_node.cpp @@ -21,10 +21,10 @@ namespace doris { -const string ScanNode::_s_bytes_read_counter = "BytesRead"; -const string ScanNode::_s_rows_read_counter = "RowsRead"; -const string ScanNode::_s_total_throughput_counter = "TotalReadThroughput"; -const string ScanNode::_s_num_disks_accessed_counter = "NumDiskAccess"; +const std::string ScanNode::_s_bytes_read_counter = "BytesRead"; +const std::string ScanNode::_s_rows_read_counter = "RowsRead"; +const std::string ScanNode::_s_total_throughput_counter = "TotalReadThroughput"; +const std::string ScanNode::_s_num_disks_accessed_counter = "NumDiskAccess"; Status ScanNode::prepare(RuntimeState* state) { RETURN_IF_ERROR(ExecNode::prepare(state)); diff --git a/be/src/exec/schema_scan_node.cpp b/be/src/exec/schema_scan_node.cpp index e937cc8dcd..25908338ab 100644 --- a/be/src/exec/schema_scan_node.cpp +++ b/be/src/exec/schema_scan_node.cpp @@ -313,7 +313,7 @@ Status SchemaScanNode::close(RuntimeState* state) { return ExecNode::close(state); } -void SchemaScanNode::debug_string(int indentation_level, stringstream* out) const { +void SchemaScanNode::debug_string(int indentation_level, std::stringstream* out) const { *out << string(indentation_level * 2, ' '); *out << "SchemaScanNode(tupleid=" << _tuple_id << " table=" << _table_name; *out << ")" << std::endl; @@ -323,7 +323,7 @@ void SchemaScanNode::debug_string(int indentation_level, stringstream* out) cons } } -Status SchemaScanNode::set_scan_ranges(const vector& scan_ranges) { +Status SchemaScanNode::set_scan_ranges(const std::vector& scan_ranges) { return Status::OK(); } diff --git a/be/src/exec/set_operation_node.cpp b/be/src/exec/set_operation_node.cpp index c3fd056f65..a377dc5594 100644 --- a/be/src/exec/set_operation_node.cpp +++ b/be/src/exec/set_operation_node.cpp @@ -85,7 +85,7 @@ Status SetOperationNode::close(RuntimeState* state) { return ExecNode::close(state); } -string SetOperationNode::get_row_output_string(TupleRow* row, const RowDescriptor& row_desc) { +std::string SetOperationNode::get_row_output_string(TupleRow* row, const RowDescriptor& row_desc) { std::stringstream out; out << "["; for (int i = 0; i < row_desc.tuple_descriptors().size(); ++i) { @@ -136,7 +136,7 @@ Status SetOperationNode::open(RuntimeState* state) { SCOPED_TIMER(_runtime_profile->total_time_counter()); RETURN_IF_CANCELLED(state); // open result expr lists. - for (const vector& exprs : _child_expr_lists) { + for (const std::vector& exprs : _child_expr_lists) { RETURN_IF_ERROR(Expr::open(exprs, state)); } // initial build hash table used for remove duplicated diff --git a/be/src/exec/union_node.cpp b/be/src/exec/union_node.cpp index 1543fffbb7..c8455bfd3e 100644 --- a/be/src/exec/union_node.cpp +++ b/be/src/exec/union_node.cpp @@ -27,7 +27,7 @@ #include "util/runtime_profile.h" #include "gen_cpp/PlanNodes_types.h" -// #include "common/names.h" +// namespace doris { @@ -76,7 +76,7 @@ Status UnionNode::prepare(RuntimeState* state) { _materialize_exprs_evaluate_timer = ADD_TIMER(_runtime_profile, "MaterializeExprsEvaluateTimer"); _codegend_union_materialize_batch_fns.resize(_child_expr_lists.size()); // Prepare const expr lists. - for (const vector& exprs : _const_expr_lists) { + for (const std::vector& exprs : _const_expr_lists) { RETURN_IF_ERROR(Expr::prepare(exprs, state, row_desc(), expr_mem_tracker())); // TODO(zc) // AddExprCtxsToFree(exprs); @@ -98,11 +98,11 @@ Status UnionNode::open(RuntimeState* state) { SCOPED_TIMER(_runtime_profile->total_time_counter()); RETURN_IF_ERROR(ExecNode::open(state)); // open const expr lists. - for (const vector& exprs : _const_expr_lists) { + for (const std::vector& exprs : _const_expr_lists) { RETURN_IF_ERROR(Expr::open(exprs, state)); } // open result expr lists. - for (const vector& exprs : _child_expr_lists) { + for (const std::vector& exprs : _child_expr_lists) { RETURN_IF_ERROR(Expr::open(exprs, state)); } diff --git a/be/src/exprs/agg_fn.cc b/be/src/exprs/agg_fn.cc index d6fff8c4be..c739f43cb5 100644 --- a/be/src/exprs/agg_fn.cc +++ b/be/src/exprs/agg_fn.cc @@ -22,8 +22,6 @@ #include "runtime/user_function_cache.h" #include "runtime/runtime_state.h" -#include "common/names.h" - using namespace doris_udf; namespace doris { @@ -44,7 +42,7 @@ AggFn::AggFn(const TExprNode& tnode, const SlotDescriptor& intermediate_slot_des DCHECK_EQ(tnode.node_type, TExprNodeType::AGG_EXPR); DCHECK_EQ(TypeDescriptor::from_thrift(tnode.type).type, TypeDescriptor::from_thrift(_fn.ret_type).type); - const string& fn_name = _fn.name.function_name; + const std::string& fn_name = _fn.name.function_name; if (fn_name == "count") { agg_op_ = COUNT; } else if (fn_name == "min") { @@ -85,7 +83,7 @@ Status AggFn::Init(const RowDescriptor& row_desc, RuntimeState* state) { (aggregate_fn.merge_fn_symbol.empty() && !aggregate_fn.is_analytic_only_fn)) { // This path is only for partially implemented builtins. DCHECK_EQ(_fn.binary_type, TFunctionBinaryType::BUILTIN); - stringstream ss; + std::stringstream ss; ss << "Function " << _fn.name.function_name << " is not implemented."; return Status::InternalError(ss.str()); } @@ -170,12 +168,12 @@ void AggFn::Close() { Expr::close(); } -void AggFn::Close(const vector& exprs) { +void AggFn::Close(const std::vector& exprs) { for (AggFn* expr : exprs) expr->Close(); } -string AggFn::DebugString() const { - stringstream out; +std::string AggFn::DebugString() const { + std::stringstream out; out << "AggFn(op=" << agg_op_; for (Expr* input_expr : children()) { out << " " << input_expr->debug_string() << ")"; @@ -184,8 +182,8 @@ string AggFn::DebugString() const { return out.str(); } -string AggFn::DebugString(const vector& agg_fns) { - stringstream out; +std::string AggFn::DebugString(const std::vector& agg_fns) { + std::stringstream out; out << "["; for (int i = 0; i < agg_fns.size(); ++i) { out << (i == 0 ? "" : " ") << agg_fns[i]->DebugString(); diff --git a/be/src/exprs/agg_fn_evaluator.cpp b/be/src/exprs/agg_fn_evaluator.cpp index e3e68e0ee5..dc0558e10a 100755 --- a/be/src/exprs/agg_fn_evaluator.cpp +++ b/be/src/exprs/agg_fn_evaluator.cpp @@ -198,7 +198,7 @@ Status AggFnEvaluator::prepare( (!_is_analytic_fn && _fn.aggregate_fn.merge_fn_symbol.empty())) { // This path is only for partially implemented builtins. DCHECK_EQ(_fn.binary_type, TFunctionBinaryType::BUILTIN); - stringstream ss; + std::stringstream ss; ss << "Function " << _fn.name.function_name << " is not implemented."; return Status::InternalError(ss.str()); } @@ -244,7 +244,7 @@ Status AggFnEvaluator::prepare( NULL)); } - vector arg_types; + std::vector arg_types; for (int j = 0; j < _input_exprs_ctxs.size(); ++j) { arg_types.push_back( AnyValUtil::column_type_to_type_desc(_input_exprs_ctxs[j]->root()->type())); @@ -265,7 +265,7 @@ Status AggFnEvaluator::open(RuntimeState* state, FunctionContext* agg_fn_ctx) { // Now that we have opened all our input exprs, it is safe to evaluate any constant // values for the UDA's FunctionContext (we cannot evaluate exprs before calling Open() // on them). - vector constant_args(_input_exprs_ctxs.size()); + std::vector constant_args(_input_exprs_ctxs.size()); for (int i = 0; i < _input_exprs_ctxs.size(); ++i) { constant_args[i] = _input_exprs_ctxs[i]->root()->get_const_val(_input_exprs_ctxs[i]); } diff --git a/be/src/exprs/expr.cpp b/be/src/exprs/expr.cpp index ee89a332c5..c7661c2b8f 100644 --- a/be/src/exprs/expr.cpp +++ b/be/src/exprs/expr.cpp @@ -279,7 +279,7 @@ Status Expr::create_expr_trees( Status Expr::create_tree_from_thrift( ObjectPool* pool, - const vector& nodes, + const std::vector& nodes, Expr* parent, int* node_idx, Expr** root_expr, @@ -892,8 +892,8 @@ Status Expr::create(const TExpr& texpr, const RowDescriptor& row_desc, RuntimeSt return Status::OK(); } -Status Expr::create(const vector& texprs, const RowDescriptor& row_desc, RuntimeState* state, - ObjectPool* pool, vector* exprs, +Status Expr::create(const std::vector& texprs, const RowDescriptor& row_desc, RuntimeState* state, + ObjectPool* pool, std::vector* exprs, const std::shared_ptr& tracker) { exprs->clear(); for (const TExpr& texpr: texprs) { @@ -910,8 +910,8 @@ Status Expr::create(const TExpr& texpr, const RowDescriptor& row_desc, return Expr::create(texpr, row_desc, state, state->obj_pool(), scalar_expr, tracker); } -Status Expr::create(const vector& texprs, const RowDescriptor& row_desc, - RuntimeState* state, vector* exprs, const std::shared_ptr& tracker) { +Status Expr::create(const std::vector& texprs, const RowDescriptor& row_desc, + RuntimeState* state, std::vector* exprs, const std::shared_ptr& tracker) { return Expr::create(texprs, row_desc, state, state->obj_pool(), exprs, tracker); } @@ -937,7 +937,7 @@ Status Expr::create_tree(const TExpr& texpr, ObjectPool* pool, Expr* root) { return Status::OK(); } -Status Expr::create_tree_internal(const vector& nodes, ObjectPool* pool, +Status Expr::create_tree_internal(const std::vector& nodes, ObjectPool* pool, Expr* root, int* child_node_idx) { // propagate error case if (*child_node_idx >= nodes.size()) { @@ -972,7 +972,7 @@ void Expr::close() { } } -void Expr::close(const vector& exprs) { +void Expr::close(const std::vector& exprs) { for (Expr* expr : exprs) expr->close(); } diff --git a/be/src/exprs/math_functions.cpp b/be/src/exprs/math_functions.cpp index 6fab6485d3..89c10ea0d7 100644 --- a/be/src/exprs/math_functions.cpp +++ b/be/src/exprs/math_functions.cpp @@ -366,7 +366,7 @@ StringVal MathFunctions::hex_string(FunctionContext* ctx, const StringVal& s) { std::stringstream ss; ss << std::hex << std::uppercase << std::setfill('0'); for (int i = 0; i < s.len; ++i) { - // setw is not sticky. stringstream only converts integral values, + // setw is not sticky. std::stringstream only converts integral values, // so a cast to int is required, but only convert the least significant byte to hex. ss << std::setw(2) << (static_cast(s.ptr[i]) & 0xFF); } diff --git a/be/src/exprs/new_agg_fn_evaluator.cc b/be/src/exprs/new_agg_fn_evaluator.cc index f1a72897c7..ca90390a94 100644 --- a/be/src/exprs/new_agg_fn_evaluator.cc +++ b/be/src/exprs/new_agg_fn_evaluator.cc @@ -36,7 +36,7 @@ #include -#include "common/names.h" + using namespace doris; using namespace doris_udf; diff --git a/be/src/exprs/scalar_fn_call.cpp b/be/src/exprs/scalar_fn_call.cpp index 7c04158038..7c6b03fc8f 100644 --- a/be/src/exprs/scalar_fn_call.cpp +++ b/be/src/exprs/scalar_fn_call.cpp @@ -120,7 +120,7 @@ Status ScalarFnCall::prepare( return status; } else { DCHECK_EQ(_fn.binary_type, TFunctionBinaryType::NATIVE); - return Status::InternalError(Substitute("Problem loading UDF '$0':\n$1", + return Status::InternalError(strings::Substitute("Problem loading UDF '$0':\n$1", _fn.name.function_name, status.GetDetail())); return status; } diff --git a/be/src/http/default_path_handlers.cpp b/be/src/http/default_path_handlers.cpp index 933e6fbc65..20dd3b09d0 100644 --- a/be/src/http/default_path_handlers.cpp +++ b/be/src/http/default_path_handlers.cpp @@ -140,7 +140,7 @@ void mem_tracker_handler(const WebPageHandler::ArgumentMap& args, std::stringstr ">Peak Consumption"; (*output) << "\n"; - vector> trackers; + std::vector> trackers; MemTracker::ListTrackers(&trackers); for (const shared_ptr& tracker : trackers) { string parent = tracker->parent() == nullptr ? "none" : tracker->parent()->label(); diff --git a/be/src/http/web_page_handler.cpp b/be/src/http/web_page_handler.cpp index 99576a10e0..417b357d08 100644 --- a/be/src/http/web_page_handler.cpp +++ b/be/src/http/web_page_handler.cpp @@ -61,7 +61,7 @@ WebPageHandler::~WebPageHandler() { void WebPageHandler::register_template_page(const std::string& path, const string& alias, const TemplatePageHandlerCallback& callback, bool is_on_nav_bar) { // Relative path which will be used to find .mustache file in _www_path - string render_path = (path == "/") ? "/home" : path; + std::string render_path = (path == "/") ? "/home" : path; auto wrapped_cb = [=](const ArgumentMap& args, std::stringstream* output) { EasyJson ej; callback(args, &ej); @@ -166,7 +166,7 @@ static const std::string kMainTemplate = R"( )"; std::string WebPageHandler::mustache_partial_tag(const std::string& path) const { - return Substitute("{{> $0.mustache}}", path); + return strings::Substitute("{{> $0.mustache}}", path); } bool WebPageHandler::static_pages_available() const { @@ -178,7 +178,7 @@ bool WebPageHandler::mustache_template_available(const std::string& path) const if (!static_pages_available()) { return false; } - return Env::Default()->path_exists(Substitute("$0/$1.mustache", _www_path, path)).ok(); + return Env::Default()->path_exists(strings::Substitute("$0/$1.mustache", _www_path, path)).ok(); } void WebPageHandler::render_main_template(const std::string& content, std::stringstream* output) { diff --git a/be/src/olap/base_compaction.cpp b/be/src/olap/base_compaction.cpp index 03bb6eb106..a29a91b753 100644 --- a/be/src/olap/base_compaction.cpp +++ b/be/src/olap/base_compaction.cpp @@ -131,7 +131,7 @@ OLAPStatus BaseCompaction::pick_rowsets_to_compact() { return OLAP_ERR_BE_NO_SUITABLE_VERSION; } -OLAPStatus BaseCompaction::_check_rowset_overlapping(const vector& rowsets) { +OLAPStatus BaseCompaction::_check_rowset_overlapping(const std::vector& rowsets) { for (auto& rs : rowsets) { if (rs->rowset_meta()->is_segments_overlapping()) { LOG(WARNING) << "There is overlapping rowset before cumulative point, " diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp index 7aa6b9fb4c..919df23008 100644 --- a/be/src/olap/compaction.cpp +++ b/be/src/olap/compaction.cpp @@ -181,8 +181,8 @@ OLAPStatus Compaction::gc_unused_rowsets() { // Two versions before and after the missing version will be saved in missing_version, // if missing_version is not null. OLAPStatus Compaction::find_longest_consecutive_version( - vector* rowsets, - vector* missing_version) { + std::vector* rowsets, + std::vector* missing_version) { if (rowsets->empty()) { return OLAP_SUCCESS; } @@ -204,7 +204,7 @@ OLAPStatus Compaction::find_longest_consecutive_version( return OLAP_SUCCESS; } -OLAPStatus Compaction::check_version_continuity(const vector& rowsets) { +OLAPStatus Compaction::check_version_continuity(const std::vector& rowsets) { RowsetSharedPtr prev_rowset = rowsets.front(); for (size_t i = 1; i < rowsets.size(); ++i) { RowsetSharedPtr rowset = rowsets[i]; diff --git a/be/src/olap/comparison_predicate.cpp b/be/src/olap/comparison_predicate.cpp index 23e9d021df..08b673f7dc 100644 --- a/be/src/olap/comparison_predicate.cpp +++ b/be/src/olap/comparison_predicate.cpp @@ -195,7 +195,7 @@ COMPARISON_PRED_COLUMN_BLOCK_EVALUATE(GreaterEqualPredicate, >=) #define COMPARISON_PRED_BITMAP_EVALUATE(CLASS, OP) \ template \ - Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const { \ + Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const { \ BitmapIndexIterator* iterator = iterators[_column_id]; \ if (iterator == nullptr) { \ return Status::OK(); \ @@ -288,18 +288,18 @@ COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_DECLARATION(GreaterPredicate) COMPARISON_PRED_COLUMN_BLOCK_EVALUATE_DECLARATION(GreaterEqualPredicate) #define COMPARISON_PRED_BITMAP_EVALUATE_DECLARATION(CLASS) \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ COMPARISON_PRED_BITMAP_EVALUATE_DECLARATION(EqualPredicate) COMPARISON_PRED_BITMAP_EVALUATE_DECLARATION(NotEqualPredicate) diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp index 86650d144c..9dbe65833d 100644 --- a/be/src/olap/data_dir.cpp +++ b/be/src/olap/data_dir.cpp @@ -97,7 +97,7 @@ DataDir::~DataDir() { Status DataDir::init() { if (!FileUtils::check_exist(_path)) { - RETURN_NOT_OK_STATUS_WITH_WARN(Status::IOError(Substitute("opendir failed, path=$0", _path)), + RETURN_NOT_OK_STATUS_WITH_WARN(Status::IOError(strings::Substitute("opendir failed, path=$0", _path)), "check file exist failed"); } @@ -123,7 +123,7 @@ Status DataDir::_init_cluster_id() { S_IRUSR | S_IWUSR | S_IRGRP | S_IWGRP); if (fd < 0 || close(fd) < 0) { RETURN_NOT_OK_STATUS_WITH_WARN( - Status::IOError(Substitute("failed to create cluster id file $0, err=$1", + Status::IOError(strings::Substitute("failed to create cluster id file $0, err=$1", cluster_id_path, errno_to_string(errno))), "create file failed"); } @@ -134,7 +134,7 @@ Status DataDir::_init_cluster_id() { fp = fopen(cluster_id_path.c_str(), "r+b"); if (fp == NULL) { RETURN_NOT_OK_STATUS_WITH_WARN( - Status::IOError(Substitute("failed to open cluster id file $0", cluster_id_path)), + Status::IOError(strings::Substitute("failed to open cluster id file $0", cluster_id_path)), "open file failed"); } @@ -143,7 +143,7 @@ Status DataDir::_init_cluster_id() { fclose(fp); fp = NULL; RETURN_NOT_OK_STATUS_WITH_WARN( - Status::IOError(Substitute("failed to flock cluster id file $0", cluster_id_path)), + Status::IOError(strings::Substitute("failed to flock cluster id file $0", cluster_id_path)), "flock file failed"); } @@ -159,7 +159,7 @@ Status DataDir::_read_cluster_id(const std::string& cluster_id_path, int32_t* cl std::fstream fs(cluster_id_path.c_str(), std::fstream::in); if (!fs.is_open()) { RETURN_NOT_OK_STATUS_WITH_WARN( - Status::IOError(Substitute("failed to open cluster id file $0", cluster_id_path)), + Status::IOError(strings::Substitute("failed to open cluster id file $0", cluster_id_path)), "open file failed"); } @@ -173,7 +173,7 @@ Status DataDir::_read_cluster_id(const std::string& cluster_id_path, int32_t* cl } else { RETURN_NOT_OK_STATUS_WITH_WARN( Status::Corruption( - Substitute("cluster id file $0 is corrupt. [id=$1 eofbit=$2 failbit=$3 badbit=$4]", + strings::Substitute("cluster id file $0 is corrupt. [id=$1 eofbit=$2 failbit=$3 badbit=$4]", cluster_id_path, tmp_cluster_id, fs.rdstate() & std::fstream::eofbit, fs.rdstate() & std::fstream::failbit, fs.rdstate() & std::fstream::badbit)), "file content is error"); @@ -189,7 +189,7 @@ Status DataDir::_init_capacity() { } else if (_capacity_bytes > disk_capacity) { RETURN_NOT_OK_STATUS_WITH_WARN( Status::InvalidArgument( - Substitute("root path $0's capacity $1 should not larger than disk capacity $2", + strings::Substitute("root path $0's capacity $1 should not larger than disk capacity $2", _path, _capacity_bytes, disk_capacity)), "init capacity failed"); } @@ -197,7 +197,7 @@ Status DataDir::_init_capacity() { std::string data_path = _path + DATA_PREFIX; if (!FileUtils::check_exist(data_path) && !FileUtils::create_dir(data_path).ok()) { RETURN_NOT_OK_STATUS_WITH_WARN( - Status::IOError(Substitute("failed to create data root path $0", data_path)), + Status::IOError(strings::Substitute("failed to create data root path $0", data_path)), "check_exist failed"); } @@ -209,7 +209,7 @@ Status DataDir::_init_file_system() { if (stat(_path.c_str(), &s) != 0) { RETURN_NOT_OK_STATUS_WITH_WARN( Status::IOError( - Substitute("stat file $0 failed, err=$1", _path, errno_to_string(errno))), + strings::Substitute("stat file $0 failed, err=$1", _path, errno_to_string(errno))), "stat file failed"); } @@ -224,7 +224,7 @@ Status DataDir::_init_file_system() { if ((mount_tablet = setmntent(kMtabPath, "r")) == NULL) { RETURN_NOT_OK_STATUS_WITH_WARN( Status::IOError( - Substitute("setmntent file $0 failed, err=$1", _path, errno_to_string(errno))), + strings::Substitute("setmntent file $0 failed, err=$1", _path, errno_to_string(errno))), "setmntent file failed"); } @@ -254,7 +254,7 @@ Status DataDir::_init_file_system() { if (!is_find) { RETURN_NOT_OK_STATUS_WITH_WARN( - Status::IOError(Substitute("file system $0 not found", _path)), + Status::IOError(strings::Substitute("file system $0 not found", _path)), "find file system failed"); } @@ -278,7 +278,7 @@ Status DataDir::_init_meta() { OLAPStatus res = _meta->init(); if (res != OLAP_SUCCESS) { RETURN_NOT_OK_STATUS_WITH_WARN( - Status::IOError(Substitute("open rocksdb failed, path=$0", _path)), + Status::IOError(strings::Substitute("open rocksdb failed, path=$0", _path)), "init OlapMeta failed"); } return Status::OK(); @@ -367,12 +367,12 @@ void DataDir::clear_tablets(std::vector* tablet_infos) { } std::string DataDir::get_absolute_shard_path(int64_t shard_id) { - return Substitute("$0$1/$2", _path, DATA_PREFIX, shard_id); + return strings::Substitute("$0$1/$2", _path, DATA_PREFIX, shard_id); } std::string DataDir::get_absolute_tablet_path(int64_t shard_id, int64_t tablet_id, int32_t schema_hash) { - return Substitute("$0/$1/$2", get_absolute_shard_path(shard_id), tablet_id, schema_hash); + return strings::Substitute("$0/$1/$2", get_absolute_shard_path(shard_id), tablet_id, schema_hash); } void DataDir::find_tablet_in_trash(int64_t tablet_id, std::vector* paths) { @@ -447,7 +447,7 @@ OLAPStatus DataDir::_convert_old_tablet() { // convert olap header and files OLAPHeaderMessage olap_header_msg; TabletMetaPB tablet_meta_pb; - vector pending_rowsets; + std::vector pending_rowsets; bool parsed = olap_header_msg.ParseFromString(value); if (!parsed) { LOG(FATAL) << "convert olap header to tablet meta failed when load olap header tablet=" @@ -514,7 +514,7 @@ OLAPStatus DataDir::remove_old_meta_and_files() { // convert olap header and files OLAPHeaderMessage olap_header_msg; TabletMetaPB tablet_meta_pb; - vector pending_rowsets; + std::vector pending_rowsets; bool parsed = olap_header_msg.ParseFromString(value); if (!parsed) { LOG(FATAL) << "convert olap header to tablet meta failed when load olap header tablet=" @@ -971,7 +971,7 @@ Status DataDir::update_capacity() { } catch (boost::filesystem::filesystem_error& e) { RETURN_NOT_OK_STATUS_WITH_WARN( Status::IOError( - Substitute("get path $0 available capacity failed, error=$1", _path, e.what())), + strings::Substitute("get path $0 available capacity failed, error=$1", _path, e.what())), "boost::filesystem::space failed"); } diff --git a/be/src/olap/delete_handler.cpp b/be/src/olap/delete_handler.cpp index 8e8e80a8dc..52935f5ef7 100644 --- a/be/src/olap/delete_handler.cpp +++ b/be/src/olap/delete_handler.cpp @@ -89,7 +89,7 @@ OLAPStatus DeleteConditionHandler::generate_delete_predicate( return OLAP_SUCCESS; } -string DeleteConditionHandler::construct_sub_predicates(const TCondition& condition) { +std::string DeleteConditionHandler::construct_sub_predicates(const TCondition& condition) { string op = condition.condition_op; if (op == "<") { op += "<"; @@ -298,7 +298,7 @@ bool DeleteHandler::is_filter_data(const int32_t data_version, // 根据语义,存储在_del_conds的删除条件应该是OR关系 // 因此,只要数据符合其中一条过滤条件,则返回true - vector::const_iterator it = _del_conds.begin(); + std::vector::const_iterator it = _del_conds.begin(); for (; it != _del_conds.end(); ++it) { if (data_version <= it->filter_version && it->del_cond->delete_conditions_eval(row)) { @@ -309,9 +309,9 @@ bool DeleteHandler::is_filter_data(const int32_t data_version, return false; } -vector DeleteHandler::get_conds_version() { - vector conds_version; - vector::const_iterator cond_iter = _del_conds.begin(); +std::vector DeleteHandler::get_conds_version() { + std::vector conds_version; + std::vector::const_iterator cond_iter = _del_conds.begin(); for (; cond_iter != _del_conds.end(); ++cond_iter) { conds_version.push_back(cond_iter->filter_version); @@ -325,7 +325,7 @@ void DeleteHandler::finalize() { return; } - vector::iterator it = _del_conds.begin(); + std::vector::iterator it = _del_conds.begin(); for (; it != _del_conds.end(); ++it) { it->del_cond->finalize(); diff --git a/be/src/olap/fs/block_id.cpp b/be/src/olap/fs/block_id.cpp index 2243b7d6bb..f5cf4f7551 100644 --- a/be/src/olap/fs/block_id.cpp +++ b/be/src/olap/fs/block_id.cpp @@ -30,8 +30,8 @@ namespace doris { const uint64_t BlockId::kInvalidId = 0; -string BlockId::join_strings(const vector& blocks) { - vector strings; +std::string BlockId::join_strings(const std::vector& blocks) { + std::vector strings; strings.reserve(blocks.size()); for (const BlockId& block : blocks) { strings.push_back(block.to_string()); diff --git a/be/src/olap/fs/file_block_manager.cpp b/be/src/olap/fs/file_block_manager.cpp index c49388593c..22103fba6e 100644 --- a/be/src/olap/fs/file_block_manager.cpp +++ b/be/src/olap/fs/file_block_manager.cpp @@ -42,7 +42,7 @@ using std::accumulate; using std::shared_ptr; using std::string; -using std::unique_ptr; + using strings::Substitute; namespace doris { @@ -137,7 +137,7 @@ FileWritableBlock::FileWritableBlock(FileBlockManager* block_manager, FileWritableBlock::~FileWritableBlock() { if (_state != CLOSED) { - WARN_IF_ERROR(abort(), Substitute("Failed to close block $0", _path)); + WARN_IF_ERROR(abort(), strings::Substitute("Failed to close block $0", _path)); } } @@ -229,7 +229,7 @@ Status FileWritableBlock::_close(SyncMode mode) { if (sync.ok()) { sync = _block_manager->_sync_metadata(_path); } - WARN_IF_ERROR(sync, Substitute("Failed to sync when closing block $0", _path)); + WARN_IF_ERROR(sync, strings::Substitute("Failed to sync when closing block $0", _path)); } Status close = _writer->close(); @@ -316,7 +316,7 @@ FileReadableBlock::FileReadableBlock(FileBlockManager* block_manager, } FileReadableBlock::~FileReadableBlock() { - WARN_IF_ERROR(close(), Substitute("Failed to close block $0", _path)); + WARN_IF_ERROR(close(), strings::Substitute("Failed to close block $0", _path)); } Status FileReadableBlock::close() { @@ -402,7 +402,7 @@ Status FileBlockManager::open() { } Status FileBlockManager::create_block(const CreateBlockOptions& opts, - unique_ptr* block) { + std::unique_ptr* block) { CHECK(!_opts.read_only); shared_ptr writer; @@ -415,7 +415,7 @@ Status FileBlockManager::create_block(const CreateBlockOptions& opts, return Status::OK(); } -Status FileBlockManager::open_block(const std::string& path, unique_ptr* block) { +Status FileBlockManager::open_block(const std::string& path, std::unique_ptr* block) { VLOG(1) << "Opening block with path at " << path; std::shared_ptr> file_handle(new OpenedFileHandle()); bool found = _file_cache->lookup(path, file_handle.get()); diff --git a/be/src/olap/in_list_predicate.cpp b/be/src/olap/in_list_predicate.cpp index 4cfaeaa5af..166fb0a854 100644 --- a/be/src/olap/in_list_predicate.cpp +++ b/be/src/olap/in_list_predicate.cpp @@ -111,7 +111,7 @@ IN_LIST_PRED_COLUMN_BLOCK_EVALUATE(NotInListPredicate, ==) #define IN_LIST_PRED_BITMAP_EVALUATE(CLASS, OP) \ template \ - Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* result) const { \ + Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* result) const { \ BitmapIndexIterator *iterator = iterators[_column_id]; \ if (iterator == nullptr) { \ return Status::OK(); \ @@ -191,17 +191,17 @@ IN_LIST_PRED_COLUMN_BLOCK_EVALUATE_DECLARATION(InListPredicate) IN_LIST_PRED_COLUMN_BLOCK_EVALUATE_DECLARATION(NotInListPredicate) #define IN_LIST_PRED_BITMAP_EVALUATE_DECLARATION(CLASS) \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ - template Status CLASS::evaluate(const Schema& schema, const vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ + template Status CLASS::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* bitmap) const; \ IN_LIST_PRED_BITMAP_EVALUATE_DECLARATION(InListPredicate) IN_LIST_PRED_BITMAP_EVALUATE_DECLARATION(NotInListPredicate) diff --git a/be/src/olap/memory/column.cpp b/be/src/olap/memory/column.cpp index ff8d98788b..d99e4ec61b 100644 --- a/be/src/olap/memory/column.cpp +++ b/be/src/olap/memory/column.cpp @@ -60,12 +60,12 @@ size_t Column::memory() const { return base_memory + delta_memory; } -string Column::debug_string() const { +std::string Column::debug_string() const { return StringPrintf("Column(%s version=%zu)", _cs.debug_string().c_str(), _versions.back().version); } -Status Column::capture_version(uint64_t version, vector* deltas, +Status Column::capture_version(uint64_t version, std::vector* deltas, uint64_t* real_version) const { uint64_t base_version = _versions[_base_idx].version; *real_version = base_version; @@ -119,7 +119,7 @@ void Column::capture_latest(vector* deltas, uint64_t* version) con Status Column::create_reader(uint64_t version, std::unique_ptr* reader) { ColumnType type = schema().type(); bool nullable = schema().is_nullable(); - vector deltas; + std::vector deltas; uint64_t real_version; RETURN_IF_ERROR(capture_version(version, &deltas, &real_version)); @@ -165,7 +165,7 @@ Status Column::create_reader(uint64_t version, std::unique_ptr* re Status Column::create_writer(std::unique_ptr* writer) { ColumnType type = schema().type(); bool nullable = schema().is_nullable(); - vector deltas; + std::vector deltas; uint64_t real_version; capture_latest(&deltas, &real_version); diff --git a/be/src/olap/null_predicate.cpp b/be/src/olap/null_predicate.cpp index eed4df51b0..af33512989 100644 --- a/be/src/olap/null_predicate.cpp +++ b/be/src/olap/null_predicate.cpp @@ -78,7 +78,7 @@ void NullPredicate::evaluate(ColumnBlock* block, uint16_t* sel, uint16_t* size) *size = new_size; } -Status NullPredicate::evaluate(const Schema& schema, const vector& iterators, +Status NullPredicate::evaluate(const Schema& schema, const std::vector& iterators, uint32_t num_rows, Roaring* roaring) const { if (iterators[_column_id] != nullptr) { Roaring null_bitmap; diff --git a/be/src/olap/olap_cond.cpp b/be/src/olap/olap_cond.cpp index 74c603fde5..766e1320a0 100644 --- a/be/src/olap/olap_cond.cpp +++ b/be/src/olap/olap_cond.cpp @@ -161,9 +161,9 @@ OLAPStatus Cond::init(const TCondition& tcond, const TabletColumn& column) { auto insert_result = operand_set.insert(f.get()); if (!insert_result.second) { LOG(WARNING) << "Duplicate operand in in-predicate.[condition=" << operand << "]"; - // Duplicated, let unique_ptr delete field + // Duplicated, let std::unique_ptr delete field } else { - // Normal case, release this unique_ptr + // Normal case, release this std::unique_ptr f.release(); } } diff --git a/be/src/olap/olap_server.cpp b/be/src/olap/olap_server.cpp index fa1aa7b049..3036e7841c 100644 --- a/be/src/olap/olap_server.cpp +++ b/be/src/olap/olap_server.cpp @@ -337,7 +337,7 @@ void StorageEngine::_compaction_tasks_producer_callback() { compaction_type = CompactionType::BASE_COMPACTION; round = 0; } - vector tablets_compaction = + std::vector tablets_compaction = _compaction_tasks_generator(compaction_type, data_dirs); if (tablets_compaction.size() == 0) { _wakeup_producer_flag = 0; @@ -363,7 +363,7 @@ void StorageEngine::_compaction_tasks_producer_callback() { _perform_cumulative_compaction(tablet); _permit_limiter.release(permits); std::unique_lock lock(_tablet_submitted_compaction_mutex); - vector::iterator it_tablet = + std::vector::iterator it_tablet = find(_tablet_submitted_compaction[tablet->data_dir()].begin(), _tablet_submitted_compaction[tablet->data_dir()].end(), tablet->tablet_id()); @@ -380,7 +380,7 @@ void StorageEngine::_compaction_tasks_producer_callback() { _perform_base_compaction(tablet); _permit_limiter.release(permits); std::unique_lock lock(_tablet_submitted_compaction_mutex); - vector::iterator it_tablet = + std::vector::iterator it_tablet = find(_tablet_submitted_compaction[tablet->data_dir()].begin(), _tablet_submitted_compaction[tablet->data_dir()].end(), tablet->tablet_id()); @@ -400,9 +400,9 @@ void StorageEngine::_compaction_tasks_producer_callback() { } } -vector StorageEngine::_compaction_tasks_generator( +std::vector StorageEngine::_compaction_tasks_generator( CompactionType compaction_type, std::vector data_dirs) { - vector tablets_compaction; + std::vector tablets_compaction; std::random_shuffle(data_dirs.begin(), data_dirs.end()); for (auto data_dir : data_dirs) { std::unique_lock lock(_tablet_submitted_compaction_mutex); diff --git a/be/src/olap/olap_snapshot_converter.cpp b/be/src/olap/olap_snapshot_converter.cpp index 0c4c159191..a312cd955f 100755 --- a/be/src/olap/olap_snapshot_converter.cpp +++ b/be/src/olap/olap_snapshot_converter.cpp @@ -95,7 +95,7 @@ OLAPStatus OlapSnapshotConverter::to_olap_header(const TabletMetaPB& tablet_meta } OLAPStatus OlapSnapshotConverter::to_tablet_meta_pb(const OLAPHeaderMessage& olap_header, - TabletMetaPB* tablet_meta_pb, vector* pending_rowsets) { + TabletMetaPB* tablet_meta_pb, std::vector* pending_rowsets) { if (olap_header.has_tablet_id()) { tablet_meta_pb->set_tablet_id(olap_header.tablet_id()); } @@ -442,7 +442,7 @@ OLAPStatus OlapSnapshotConverter::to_alter_tablet_pb(const SchemaChangeStatusMes // from olap header to tablet meta OLAPStatus OlapSnapshotConverter::to_new_snapshot(const OLAPHeaderMessage& olap_header, const string& old_data_path_prefix, const string& new_data_path_prefix, TabletMetaPB* tablet_meta_pb, - vector* pending_rowsets, bool is_startup) { + std::vector* pending_rowsets, bool is_startup) { RETURN_NOT_OK(to_tablet_meta_pb(olap_header, tablet_meta_pb, pending_rowsets)); TabletSchema tablet_schema; diff --git a/be/src/olap/options.cpp b/be/src/olap/options.cpp index ae35f1c8a4..a7ddd44528 100644 --- a/be/src/olap/options.cpp +++ b/be/src/olap/options.cpp @@ -49,7 +49,7 @@ static std::string to_upper(const std::string& str) { // format 1: /home/disk1/palo.HDD,50 // format 2: /home/disk1/palo,medium:ssd,capacity:50 OLAPStatus parse_root_path(const string& root_path, StorePath* path) { - vector tmp_vec = strings::Split(root_path, ",", strings::SkipWhitespace()); + std::vector tmp_vec = strings::Split(root_path, ",", strings::SkipWhitespace()); // parse root path name StripWhiteSpace(&tmp_vec[0]); @@ -131,8 +131,8 @@ OLAPStatus parse_root_path(const string& root_path, StorePath* path) { return OLAP_SUCCESS; } -OLAPStatus parse_conf_store_paths(const string& config_path, vector* paths) { - vector path_vec = strings::Split(config_path, ";", strings::SkipWhitespace()); +OLAPStatus parse_conf_store_paths(const string& config_path, std::vector* paths) { + std::vector path_vec = strings::Split(config_path, ";", strings::SkipWhitespace()); for (auto& item : path_vec) { StorePath path; auto res = parse_root_path(item, &path); diff --git a/be/src/olap/push_handler.cpp b/be/src/olap/push_handler.cpp index 0ca7235897..e764e853eb 100644 --- a/be/src/olap/push_handler.cpp +++ b/be/src/olap/push_handler.cpp @@ -56,13 +56,13 @@ namespace doris { // very useful in rollup action. OLAPStatus PushHandler::process_streaming_ingestion( TabletSharedPtr tablet, const TPushReq& request, PushType push_type, - vector* tablet_info_vec) { + std::vector* tablet_info_vec) { LOG(INFO) << "begin to realtime push. tablet=" << tablet->full_name() << ", transaction_id=" << request.transaction_id; OLAPStatus res = OLAP_SUCCESS; _request = request; - vector tablet_vars(1); + std::vector tablet_vars(1); tablet_vars[0].tablet = tablet; res = _do_streaming_ingestion(tablet, request, push_type, &tablet_vars, tablet_info_vec); @@ -82,7 +82,7 @@ OLAPStatus PushHandler::process_streaming_ingestion( OLAPStatus PushHandler::_do_streaming_ingestion( TabletSharedPtr tablet, const TPushReq& request, PushType push_type, - vector* tablet_vars, + std::vector* tablet_vars, std::vector* tablet_info_vec) { // add transaction in engine, then check sc status // lock, prevent sc handler checking transaction concurrently @@ -251,8 +251,8 @@ OLAPStatus PushHandler::_do_streaming_ingestion( return res; } -void PushHandler::_get_tablet_infos(const vector& tablet_vars, - vector* tablet_info_vec) { +void PushHandler::_get_tablet_infos(const std::vector& tablet_vars, + std::vector* tablet_info_vec) { for (const TabletVars& tablet_var : tablet_vars) { if (tablet_var.tablet.get() == NULL) { continue; @@ -1052,7 +1052,7 @@ void PushBrokerReader::print_profile() { LOG(INFO) << ss.str(); } -string PushHandler::_debug_version_list(const Versions& versions) const { +std::string PushHandler::_debug_version_list(const Versions& versions) const { std::ostringstream txt; txt << "Versions: "; diff --git a/be/src/olap/row_cursor.cpp b/be/src/olap/row_cursor.cpp index 9fad7ba7c5..61433d443a 100644 --- a/be/src/olap/row_cursor.cpp +++ b/be/src/olap/row_cursor.cpp @@ -102,7 +102,7 @@ OLAPStatus RowCursor::init(const std::vector& schema, size_t colum } OLAPStatus RowCursor::init(const TabletSchema& schema, - const vector& columns) { + const std::vector& columns) { RETURN_NOT_OK(_init(schema.columns(), columns)); return OLAP_SUCCESS; } @@ -253,8 +253,8 @@ OlapTuple RowCursor::to_tuple() const { return tuple; } -string RowCursor::to_string() const { - string result; +std::string RowCursor::to_string() const { + std::string result; size_t i = 0; for (auto cid : _schema->column_ids()) { if (i++ > 0) { diff --git a/be/src/olap/rowset/alpha_rowset.cpp b/be/src/olap/rowset/alpha_rowset.cpp index bb9e6a994f..a9f5a4cf83 100644 --- a/be/src/olap/rowset/alpha_rowset.cpp +++ b/be/src/olap/rowset/alpha_rowset.cpp @@ -84,7 +84,7 @@ OLAPStatus AlphaRowset::remove() { void AlphaRowset::make_visible_extra(Version version, VersionHash version_hash) { AlphaRowsetMetaSharedPtr alpha_rowset_meta = std::dynamic_pointer_cast(_rowset_meta); - vector published_segment_groups; + std::vector published_segment_groups; alpha_rowset_meta->get_segment_groups(&published_segment_groups); int32_t segment_group_idx = 0; for (auto& segment_group : _segment_groups) { @@ -167,7 +167,7 @@ OLAPStatus AlphaRowset::split_range( const RowCursor& start_key, const RowCursor& end_key, uint64_t request_block_row_count, - vector* ranges) { + std::vector* ranges) { EntrySlice entry; RowBlockPosition start_pos; RowBlockPosition end_pos; diff --git a/be/src/olap/rowset/segment_v2/binary_dict_page.cpp b/be/src/olap/rowset/segment_v2/binary_dict_page.cpp index a12eb34978..82866e0c43 100644 --- a/be/src/olap/rowset/segment_v2/binary_dict_page.cpp +++ b/be/src/olap/rowset/segment_v2/binary_dict_page.cpp @@ -78,7 +78,7 @@ Status BinaryDictPageBuilder::add(const uint8_t* vals, size_t* count) { if (src->size > 0) { char* item_mem = (char*)_pool.allocate(src->size); if (item_mem == nullptr) { - return Status::MemoryAllocFailed(Substitute("memory allocate failed, size:$0", src->size)); + return Status::MemoryAllocFailed(strings::Substitute("memory allocate failed, size:$0", src->size)); } dict_item.relocate(item_mem); } @@ -190,7 +190,7 @@ BinaryDictPageDecoder::BinaryDictPageDecoder(Slice data, const PageDecoderOption Status BinaryDictPageDecoder::init() { CHECK(!_parsed); if (_data.size < BINARY_DICT_PAGE_HEADER_SIZE) { - return Status::Corruption(Substitute("invalid data size:$0, header size:$1", + return Status::Corruption(strings::Substitute("invalid data size:$0, header size:$1", _data.size, BINARY_DICT_PAGE_HEADER_SIZE)); } size_t type = decode_fixed32_le((const uint8_t*)&_data.data[0]); @@ -213,7 +213,7 @@ Status BinaryDictPageDecoder::init() { _data_page_decoder.reset(new BinaryPlainPageDecoder(_data, _options)); } else { LOG(WARNING) << "invalid encoding type:" << _encoding_type; - return Status::Corruption(Substitute("invalid encoding type:$0", _encoding_type)); + return Status::Corruption(strings::Substitute("invalid encoding type:$0", _encoding_type)); } RETURN_IF_ERROR(_data_page_decoder->init()); @@ -257,7 +257,7 @@ Status BinaryDictPageDecoder::next_batch(size_t* n, ColumnBlockView* dst) { if (element.size > 0) { char* destination = (char*)dst->column_block()->pool()->allocate(element.size); if (destination == nullptr) { - return Status::MemoryAllocFailed(Substitute( + return Status::MemoryAllocFailed(strings::Substitute( "memory allocate failed, size:$0", element.size)); } element.relocate(destination); diff --git a/be/src/olap/rowset/segment_v2/binary_prefix_page.cpp b/be/src/olap/rowset/segment_v2/binary_prefix_page.cpp index c26602a04d..d812d8d256 100644 --- a/be/src/olap/rowset/segment_v2/binary_prefix_page.cpp +++ b/be/src/olap/rowset/segment_v2/binary_prefix_page.cpp @@ -122,7 +122,7 @@ Status BinaryPrefixPageDecoder::_read_next_value() { uint32_t non_shared_len; auto data_ptr = _decode_value_lengths(_next_ptr, &shared_len, &non_shared_len); if (data_ptr == nullptr) { - return Status::Corruption(Substitute("Failed to decode value at position $0", _cur_pos)); + return Status::Corruption(strings::Substitute("Failed to decode value at position $0", _cur_pos)); } _current_value.resize(shared_len); _current_value.append(data_ptr, non_shared_len); @@ -214,7 +214,7 @@ Status BinaryPrefixPageDecoder::_read_next_value_to_output(Slice prev, MemPool* uint32_t non_shared_len; auto data_ptr = _decode_value_lengths(_next_ptr, &shared_len, &non_shared_len); if (data_ptr == nullptr) { - return Status::Corruption(Substitute("Failed to decode value at position $0", _cur_pos)); + return Status::Corruption(strings::Substitute("Failed to decode value at position $0", _cur_pos)); } output->size = shared_len + non_shared_len; @@ -234,7 +234,7 @@ Status BinaryPrefixPageDecoder::_copy_current_to_output(MemPool* mem_pool, Slice output->data = (char*) mem_pool->allocate(output->size); if (output->data == nullptr) { return Status::MemoryAllocFailed( - Substitute("failed to allocate $0 bytes", output->size)); + strings::Substitute("failed to allocate $0 bytes", output->size)); } memcpy(output->data, _current_value.data(), output->size); } diff --git a/be/src/olap/rowset/segment_v2/column_reader.cpp b/be/src/olap/rowset/segment_v2/column_reader.cpp index 7da615068d..0307720fb5 100644 --- a/be/src/olap/rowset/segment_v2/column_reader.cpp +++ b/be/src/olap/rowset/segment_v2/column_reader.cpp @@ -86,7 +86,7 @@ ColumnReader::~ColumnReader() = default; Status ColumnReader::init() { _type_info = get_type_info(&_meta); if (_type_info == nullptr) { - return Status::NotSupported(Substitute("unsupported typeinfo, type=$0", _meta.type())); + return Status::NotSupported(strings::Substitute("unsupported typeinfo, type=$0", _meta.type())); } RETURN_IF_ERROR(EncodingInfo::get(_type_info, _meta.encoding(), &_encoding_info)); RETURN_IF_ERROR(get_block_compression_codec(_meta.compression(), &_compress_codec)); @@ -107,12 +107,12 @@ Status ColumnReader::init() { _bf_index_meta = &index_meta.bloom_filter_index(); break; default: - return Status::Corruption(Substitute( + return Status::Corruption(strings::Substitute( "Bad file $0: invalid column index type $1", _file_name, index_meta.type())); } } if (_ordinal_index_meta == nullptr) { - return Status::Corruption(Substitute( + return Status::Corruption(strings::Substitute( "Bad file $0: missing ordinal index for column $1", _file_name, _meta.column_id())); } return Status::OK(); @@ -312,7 +312,7 @@ Status ColumnReader::seek_at_or_before(ordinal_t ordinal, OrdinalPageIndexIterat RETURN_IF_ERROR(_ensure_index_loaded()); *iter = _ordinal_index->seek_at_or_before(ordinal); if (!iter->valid()) { - return Status::NotFound(Substitute("Failed to seek to ordinal $0, ", ordinal)); + return Status::NotFound(strings::Substitute("Failed to seek to ordinal $0, ", ordinal)); } return Status::OK(); } diff --git a/be/src/olap/rowset/segment_v2/column_writer.cpp b/be/src/olap/rowset/segment_v2/column_writer.cpp index cb7420fa66..abc48ab6d2 100644 --- a/be/src/olap/rowset/segment_v2/column_writer.cpp +++ b/be/src/olap/rowset/segment_v2/column_writer.cpp @@ -193,7 +193,7 @@ Status ScalarColumnWriter::init() { RETURN_IF_ERROR(_encoding_info->create_page_builder(opts, &page_builder)); if (page_builder == nullptr) { return Status::NotSupported( - Substitute("Failed to create page builder for type $0 and encoding $1", + strings::Substitute("Failed to create page builder for type $0 and encoding $1", get_field()->type(), _opts.meta->encoding())); } // should store more concrete encoding type instead of DEFAULT_ENCODING @@ -372,7 +372,7 @@ Status ScalarColumnWriter::finish_current_page() { } // build data page body : encoded values + [nullmap] - vector body; + std::vector body; OwnedSlice encoded_values = _page_builder->finish(); _page_builder->reset(); body.push_back(encoded_values.slice()); diff --git a/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp b/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp index 4afc319f2a..be3153e553 100644 --- a/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp +++ b/be/src/olap/rowset/segment_v2/indexed_column_reader.cpp @@ -33,7 +33,7 @@ Status IndexedColumnReader::load(bool use_page_cache, bool kept_in_memory) { _type_info = get_type_info((FieldType)_meta.data_type()); if (_type_info == nullptr) { - return Status::NotSupported(Substitute("unsupported typeinfo, type=$0", _meta.data_type())); + return Status::NotSupported(strings::Substitute("unsupported typeinfo, type=$0", _meta.data_type())); } RETURN_IF_ERROR(EncodingInfo::get(_type_info, _meta.encoding(), &_encoding_info)); RETURN_IF_ERROR(get_block_compression_codec(_meta.compression(), &_compress_codec)); diff --git a/be/src/olap/rowset/segment_v2/page_io.cpp b/be/src/olap/rowset/segment_v2/page_io.cpp index 4871b90ddc..3531e3174c 100644 --- a/be/src/olap/rowset/segment_v2/page_io.cpp +++ b/be/src/olap/rowset/segment_v2/page_io.cpp @@ -136,7 +136,7 @@ Status PageIO::read_and_decompress_page(const PageReadOptions& opts, // every page contains 4 bytes footer length and 4 bytes checksum const uint32_t page_size = opts.page_pointer.size; if (page_size < 8) { - return Status::Corruption(Substitute("Bad page: too small size ($0)", page_size)); + return Status::Corruption(strings::Substitute("Bad page: too small size ($0)", page_size)); } // hold compressed page at first, reset to decompressed page later @@ -152,7 +152,7 @@ Status PageIO::read_and_decompress_page(const PageReadOptions& opts, uint32_t expect = decode_fixed32_le((uint8_t*) page_slice.data + page_slice.size - 4); uint32_t actual = crc32c::Value(page_slice.data, page_slice.size - 4); if (expect != actual) { - return Status::Corruption(Substitute( + return Status::Corruption(strings::Substitute( "Bad page: checksum mismatch (actual=$0 vs expect=$1)", actual, expect)); } } @@ -179,7 +179,7 @@ Status PageIO::read_and_decompress_page(const PageReadOptions& opts, Slice decompressed_body(decompressed_page.get(), footer->uncompressed_size()); RETURN_IF_ERROR(opts.codec->decompress(compressed_body, &decompressed_body)); if (decompressed_body.size != footer->uncompressed_size()) { - return Status::Corruption(Substitute( + return Status::Corruption(strings::Substitute( "Bad page: record uncompressed size=$0 vs real decompressed size=$1", footer->uncompressed_size(), decompressed_body.size)); } diff --git a/be/src/olap/rowset/segment_v2/segment.cpp b/be/src/olap/rowset/segment_v2/segment.cpp index 73fb8ae490..24487f7598 100644 --- a/be/src/olap/rowset/segment_v2/segment.cpp +++ b/be/src/olap/rowset/segment_v2/segment.cpp @@ -97,7 +97,7 @@ Status Segment::_parse_footer() { RETURN_IF_ERROR(rblock->size(&file_size)); if (file_size < 12) { - return Status::Corruption(Substitute("Bad segment file $0: file size $1 < 12", _fname, file_size)); + return Status::Corruption(strings::Substitute("Bad segment file $0: file size $1 < 12", _fname, file_size)); } uint8_t fixed_buf[12]; @@ -105,14 +105,14 @@ Status Segment::_parse_footer() { // validate magic number if (memcmp(fixed_buf + 8, k_segment_magic, k_segment_magic_length) != 0) { - return Status::Corruption(Substitute("Bad segment file $0: magic number not match", _fname)); + return Status::Corruption(strings::Substitute("Bad segment file $0: magic number not match", _fname)); } // read footer PB uint32_t footer_length = decode_fixed32_le(fixed_buf); if (file_size < 12 + footer_length) { return Status::Corruption( - Substitute("Bad segment file $0: file size $1 < $2", _fname, file_size, 12 + footer_length)); + strings::Substitute("Bad segment file $0: file size $1 < $2", _fname, file_size, 12 + footer_length)); } std::string footer_buf; footer_buf.resize(footer_length); @@ -123,13 +123,13 @@ Status Segment::_parse_footer() { uint32_t actual_checksum = crc32c::Value(footer_buf.data(), footer_buf.size()); if (actual_checksum != expect_checksum) { return Status::Corruption( - Substitute("Bad segment file $0: footer checksum not match, actual=$1 vs expect=$2", + strings::Substitute("Bad segment file $0: footer checksum not match, actual=$1 vs expect=$2", _fname, actual_checksum, expect_checksum)); } // deserialize footer PB if (!_footer.ParseFromString(footer_buf)) { - return Status::Corruption(Substitute("Bad segment file $0: failed to parse SegmentFooterPB", _fname)); + return Status::Corruption(strings::Substitute("Bad segment file $0: failed to parse SegmentFooterPB", _fname)); } return Status::OK(); } diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp index a85cce4418..c16ef099f0 100644 --- a/be/src/olap/schema_change.cpp +++ b/be/src/olap/schema_change.cpp @@ -421,7 +421,7 @@ OLAPStatus RowBlockChanger::change_row_block(const RowBlock* ref_block, int32_t // 对于没有filter的来说,相当于全部设置为1后留下 const uint32_t row_num = ref_block->row_block_info().row_num; // (0表示过滤掉不要,1表示要,过程中2表示此row要切后续不需要再比较其他列) - vector is_data_left_vec(row_num, 1); + std::vector is_data_left_vec(row_num, 1); // 一行一行地进行比较 for (size_t row_index = 0; row_index < row_num; ++row_index) { @@ -686,7 +686,7 @@ bool RowBlockSorter::sort(RowBlock** row_block) { } RowBlock* temp = nullptr; - vector row_cursor_list((*row_block)->row_block_info().row_num, nullptr); + std::vector row_cursor_list((*row_block)->row_block_info().row_num, nullptr); // create an list of row cursor as long as the number of rows in data block. for (size_t i = 0; i < (*row_block)->row_block_info().row_num; ++i) { @@ -797,7 +797,7 @@ RowBlockMerger::RowBlockMerger(TabletSharedPtr tablet) : _tablet(tablet) {} RowBlockMerger::~RowBlockMerger() {} -bool RowBlockMerger::merge(const vector& row_block_arr, RowsetWriter* rowset_writer, +bool RowBlockMerger::merge(const std::vector& row_block_arr, RowsetWriter* rowset_writer, uint64_t* merged_rows) { uint64_t tmp_merged_rows = 0; RowCursor row_cursor; @@ -866,7 +866,7 @@ MERGE_ERR: return false; } -bool RowBlockMerger::_make_heap(const vector& row_block_arr) { +bool RowBlockMerger::_make_heap(const std::vector& row_block_arr) { for (auto row_block : row_block_arr) { MergeElement element; element.row_block = row_block; @@ -1126,11 +1126,11 @@ OLAPStatus SchemaChangeWithSorting::process(RowsetReaderSharedPtr rowset_reader, // for internal sorting RowBlock* new_row_block = nullptr; - vector row_block_arr; + std::vector row_block_arr; // for external sorting // src_rowsets to store the rowset generated by internal sorting - vector src_rowsets; + std::vector src_rowsets; _temp_delta_versions.first = _temp_delta_versions.second; @@ -1300,7 +1300,7 @@ SORTING_PROCESS_ERR: return res; } -bool SchemaChangeWithSorting::_internal_sorting(const vector& row_block_arr, +bool SchemaChangeWithSorting::_internal_sorting(const std::vector& row_block_arr, const Version& version, VersionHash version_hash, TabletSharedPtr new_tablet, RowsetTypePB new_rowset_type, @@ -1346,7 +1346,7 @@ bool SchemaChangeWithSorting::_internal_sorting(const vector& row_blo bool SchemaChangeWithSorting::_external_sorting(vector& src_rowsets, RowsetWriter* rowset_writer, TabletSharedPtr new_tablet) { - vector rs_readers; + std::vector rs_readers; for (auto& rowset : src_rowsets) { RowsetReaderSharedPtr rs_reader; auto res = rowset->create_reader(&rs_reader); @@ -1452,8 +1452,8 @@ OLAPStatus SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletRe // check if the tablet has alter task // if it has alter task, it means it is under old alter process - vector versions_to_be_changed; - vector rs_readers; + std::vector versions_to_be_changed; + std::vector rs_readers; // delete handlers for new tablet DeleteHandler delete_handler; std::vector return_columns; @@ -1483,8 +1483,8 @@ OLAPStatus SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletRe // remove all data from new tablet, prevent to rewrite data(those double pushed when wait) LOG(INFO) << "begin to remove all data from new tablet to prevent rewrite." << " new_tablet=" << new_tablet->full_name(); - vector rowsets_to_delete; - vector new_tablet_versions; + std::vector rowsets_to_delete; + std::vector new_tablet_versions; new_tablet->list_versions(&new_tablet_versions); for (auto& version : new_tablet_versions) { if (version.second <= max_rowset->end_version()) { @@ -1752,14 +1752,14 @@ SCHEMA_VERSION_CONVERT_ERR: } OLAPStatus SchemaChangeHandler::_get_versions_to_be_changed( - TabletSharedPtr base_tablet, vector* versions_to_be_changed) { + TabletSharedPtr base_tablet, std::vector* versions_to_be_changed) { RowsetSharedPtr rowset = base_tablet->rowset_with_max_version(); if (rowset == nullptr) { LOG(WARNING) << "Tablet has no version. base_tablet=" << base_tablet->full_name(); return OLAP_ERR_ALTER_DELTA_DOES_NOT_EXISTS; } - vector span_versions; + std::vector span_versions; RETURN_NOT_OK(base_tablet->capture_consistent_versions(Version(0, rowset->version().second), &span_versions)); versions_to_be_changed->insert(versions_to_be_changed->end(), @@ -1771,7 +1771,7 @@ OLAPStatus SchemaChangeHandler::_get_versions_to_be_changed( OLAPStatus SchemaChangeHandler::_add_alter_task(AlterTabletType alter_tablet_type, TabletSharedPtr base_tablet, TabletSharedPtr new_tablet, - const vector& versions_to_be_changed) { + const std::vector& versions_to_be_changed) { // check new tablet exists, // prevent to set base's status after new's dropping (clear base's status) if (StorageEngine::instance()->tablet_manager()->get_tablet( @@ -1786,7 +1786,7 @@ OLAPStatus SchemaChangeHandler::_add_alter_task(AlterTabletType alter_tablet_typ versions_to_be_changed, alter_tablet_type); base_tablet->save_meta(); new_tablet->add_alter_task(base_tablet->tablet_id(), base_tablet->schema_hash(), - vector(), // empty versions + std::vector(), // empty versions alter_tablet_type); new_tablet->save_meta(); LOG(INFO) << "successfully add alter task to both base and new"; diff --git a/be/src/olap/short_key_index.cpp b/be/src/olap/short_key_index.cpp index 6c6cf6812c..da5dd58117 100644 --- a/be/src/olap/short_key_index.cpp +++ b/be/src/olap/short_key_index.cpp @@ -58,7 +58,7 @@ Status ShortKeyIndexDecoder::parse(const Slice& body, const segment_v2::ShortKey // check if body size match footer's information if (body.size != (_footer.key_bytes() + _footer.offset_bytes())) { return Status::Corruption( - Substitute("Index size not match, need=$0, real=$1", + strings::Substitute("Index size not match, need=$0, real=$1", _footer.key_bytes() + _footer.offset_bytes(), body.size)); } diff --git a/be/src/olap/snapshot_manager.cpp b/be/src/olap/snapshot_manager.cpp index 10173a2cf1..2cbdb714b3 100644 --- a/be/src/olap/snapshot_manager.cpp +++ b/be/src/olap/snapshot_manager.cpp @@ -272,7 +272,7 @@ OLAPStatus SnapshotManager::_calc_snapshot_id_path( return res; } - stringstream snapshot_id_path_stream; + std::stringstream snapshot_id_path_stream; MutexLock auto_lock(&_snapshot_mutex); // will automatically unlock when function return. snapshot_id_path_stream << tablet->data_dir()->path() << SNAPSHOT_PREFIX << "/" << time_str << "." << _snapshot_base_id++ @@ -283,10 +283,10 @@ OLAPStatus SnapshotManager::_calc_snapshot_id_path( // location: /path/to/data/DATA_PREFIX/shard_id // return: /path/to/data/DATA_PREFIX/shard_id/tablet_id/schema_hash -string SnapshotManager::get_schema_hash_full_path( +std::string SnapshotManager::get_schema_hash_full_path( const TabletSharedPtr& ref_tablet, const string& location) const { - stringstream schema_full_path_stream; + std::stringstream schema_full_path_stream; schema_full_path_stream << location << "/" << ref_tablet->tablet_id() << "/" << ref_tablet->schema_hash(); @@ -295,10 +295,10 @@ string SnapshotManager::get_schema_hash_full_path( return schema_full_path; } -string SnapshotManager::_get_header_full_path( +std::string SnapshotManager::_get_header_full_path( const TabletSharedPtr& ref_tablet, const std::string& schema_hash_path) const { - stringstream header_name_stream; + std::stringstream header_name_stream; header_name_stream << schema_hash_path << "/" << ref_tablet->tablet_id() << ".hdr"; return header_name_stream.str(); } @@ -364,7 +364,7 @@ OLAPStatus SnapshotManager::_create_snapshot_files( res = OLAP_ERR_MALLOC_ERROR; break; } - vector consistent_rowsets; + std::vector consistent_rowsets; if (request.__isset.missing_version) { ReadLock rdlock(ref_tablet->get_header_lock_ptr()); for (int64_t missed_version : request.missing_version) { @@ -419,7 +419,7 @@ OLAPStatus SnapshotManager::_create_snapshot_files( ref_tablet->generate_tablet_meta_copy_unlocked(new_tablet_meta); } - vector rs_metas; + std::vector rs_metas; for (auto& rs : consistent_rowsets) { res = rs->link_files_to(schema_full_path, rs->rowset_id()); if (res != OLAP_SUCCESS) { break; } @@ -519,7 +519,7 @@ OLAPStatus SnapshotManager::_create_snapshot_files( } OLAPStatus SnapshotManager::_convert_beta_rowsets_to_alpha(const TabletMetaSharedPtr& new_tablet_meta, - const vector& rowset_metas, const std::string& dst_path, bool is_incremental) { + const std::vector& rowset_metas, const std::string& dst_path, bool is_incremental) { OLAPStatus res = OLAP_SUCCESS; RowsetConverter rowset_converter(new_tablet_meta); std::vector new_rowset_metas; diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp index be3fbf1f91..905699ca33 100644 --- a/be/src/olap/storage_engine.cpp +++ b/be/src/olap/storage_engine.cpp @@ -212,7 +212,7 @@ Status StorageEngine::_init_store_map() { for (auto store : tmp_stores) { delete store; } - return Status::InternalError(Substitute("init path failed, error=$0", error_msg)); + return Status::InternalError(strings::Substitute("init path failed, error=$0", error_msg)); } for (auto store : tmp_stores) { @@ -247,7 +247,7 @@ Status StorageEngine::_judge_and_update_effective_cluster_id(int32_t cluster_id) } else { if (cluster_id != _effective_cluster_id) { RETURN_NOT_OK_STATUS_WITH_WARN( - Status::Corruption(Substitute("multiple cluster ids is not equal. one=$0, other=", + Status::Corruption(strings::Substitute("multiple cluster ids is not equal. one=$0, other=", _effective_cluster_id, cluster_id)), "cluster id not equal"); } @@ -380,7 +380,7 @@ Status StorageEngine::_check_all_root_path_cluster_id() { cluster_id = tmp_cluster_id; } else { RETURN_NOT_OK_STATUS_WITH_WARN( - Status::Corruption(Substitute("multiple cluster ids is not equal. one=$0, other=", + Status::Corruption(strings::Substitute("multiple cluster ids is not equal. one=$0, other=", cluster_id, tmp_cluster_id)), "cluster id not equal"); } @@ -455,7 +455,7 @@ static bool too_many_disks_are_failed(uint32_t unused_num, uint32_t total_num) { } bool StorageEngine::_delete_tablets_on_unused_root_path() { - vector tablet_info_vec; + std::vector tablet_info_vec; uint32_t unused_root_path_num = 0; uint32_t total_root_path_num = 0; @@ -542,7 +542,7 @@ void StorageEngine::clear_transaction_task(const TTransactionId transaction_id) } void StorageEngine::clear_transaction_task(const TTransactionId transaction_id, - const vector& partition_ids) { + const std::vector& partition_ids) { LOG(INFO) << "begin to clear transaction task. transaction_id=" << transaction_id; for (const TPartitionId& partition_id : partition_ids) { @@ -881,7 +881,7 @@ OLAPStatus StorageEngine::obtain_shard_path( return res; } - stringstream root_path_stream; + std::stringstream root_path_stream; root_path_stream << stores[0]->path() << DATA_PREFIX << "/" << shard; *shard_path = root_path_stream.str(); *store = stores[0]; @@ -916,7 +916,7 @@ OLAPStatus StorageEngine::load_header( } } - stringstream schema_hash_path_stream; + std::stringstream schema_hash_path_stream; schema_hash_path_stream << shard_path << "/" << request.tablet_id << "/" << request.schema_hash; @@ -966,10 +966,10 @@ OLAPStatus StorageEngine::execute_task(EngineTask* task) { // 2. do prepare work // 3. release wlock { - vector tablet_infos; + std::vector tablet_infos; task->get_related_tablets(&tablet_infos); sort(tablet_infos.begin(), tablet_infos.end()); - vector related_tablets; + std::vector related_tablets; for (TabletInfo& tablet_info : tablet_infos) { TabletSharedPtr tablet = _tablet_manager->get_tablet( tablet_info.tablet_id, tablet_info.schema_hash); @@ -1001,11 +1001,11 @@ OLAPStatus StorageEngine::execute_task(EngineTask* task) { // 2. do finish work // 3. release wlock { - vector tablet_infos; + std::vector tablet_infos; // related tablets may be changed after execute task, so that get them here again task->get_related_tablets(&tablet_infos); sort(tablet_infos.begin(), tablet_infos.end()); - vector related_tablets; + std::vector related_tablets; for (TabletInfo& tablet_info : tablet_infos) { TabletSharedPtr tablet = _tablet_manager->get_tablet( tablet_info.tablet_id, tablet_info.schema_hash); diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp index 8529892e0e..e34a764a85 100644 --- a/be/src/olap/tablet.cpp +++ b/be/src/olap/tablet.cpp @@ -150,8 +150,8 @@ void Tablet::save_meta() { } OLAPStatus Tablet::revise_tablet_meta( - const vector& rowsets_to_clone, - const vector& versions_to_delete) { + const std::vector& rowsets_to_clone, + const std::vector& versions_to_delete) { LOG(INFO) << "begin to clone data to tablet. tablet=" << full_name() << ", rowsets_to_clone=" << rowsets_to_clone.size() << ", versions_to_delete_size=" << versions_to_delete.size(); @@ -231,7 +231,7 @@ OLAPStatus Tablet::add_rowset(RowsetSharedPtr rowset, bool need_persist) { _rs_version_map[rowset->version()] = rowset; _timestamped_version_tracker.add_version(rowset->version()); - vector rowsets_to_delete; + std::vector rowsets_to_delete; // yiguolei: temp code, should remove the rowset contains by this rowset // but it should be removed in multi path version for (auto& it : _rs_version_map) { @@ -257,14 +257,14 @@ OLAPStatus Tablet::add_rowset(RowsetSharedPtr rowset, bool need_persist) { return OLAP_SUCCESS; } -void Tablet::modify_rowsets(const vector& to_add, - const vector& to_delete) { +void Tablet::modify_rowsets(const std::vector& to_add, + const std::vector& to_delete) { // the compaction process allow to compact the single version, eg: version[4-4]. // this kind of "single version compaction" has same "input version" and "output version". // which means "to_add->version()" equals to "to_delete->version()". // So we should delete the "to_delete" before adding the "to_add", // otherwise, the "to_add" will be deleted from _rs_version_map, eventually. - vector rs_metas_to_delete; + std::vector rs_metas_to_delete; for (auto& rs : to_delete) { rs_metas_to_delete.push_back(rs->rowset_meta()); _rs_version_map.erase(rs->version()); @@ -273,7 +273,7 @@ void Tablet::modify_rowsets(const vector& to_add, _stale_rs_version_map[rs->version()] = rs; } - vector rs_metas_to_add; + std::vector rs_metas_to_add; for (auto& rs : to_add) { rs_metas_to_add.push_back(rs->rowset_meta()); _rs_version_map[rs->version()] = rs; @@ -396,7 +396,7 @@ void Tablet::_delete_stale_rowset_by_version(const Version& version) { void Tablet::delete_expired_inc_rowsets() { int64_t now = UnixSeconds(); - vector> expired_versions; + std::vector> expired_versions; WriteLock wrlock(&_meta_lock); for (auto& rs_meta : _tablet_meta->all_inc_rs_metas()) { double diff = ::difftime(now, rs_meta->creation_time()); @@ -426,7 +426,7 @@ void Tablet::delete_expired_inc_rowsets() { void Tablet::delete_expired_stale_rowset() { int64_t now = UnixSeconds(); - vector> expired_versions; + std::vector> expired_versions; WriteLock wrlock(&_meta_lock); // Compute the end time to delete rowsets, when a expired rowset createtime less then this time, it will be deleted. double expired_stale_sweep_endtime = ::difftime(now, config::tablet_rowset_stale_sweep_time_sec); @@ -568,7 +568,7 @@ void Tablet::delete_expired_stale_rowset() { } OLAPStatus Tablet::capture_consistent_versions(const Version& spec_version, - vector* version_path) const { + std::vector* version_path) const { // OLAPStatus status = _rs_graph.capture_consistent_versions(spec_version, version_path); OLAPStatus status = _timestamped_version_tracker.capture_consistent_versions(spec_version, version_path); @@ -614,15 +614,15 @@ void Tablet::list_versions(vector* versions) const { } OLAPStatus Tablet::capture_consistent_rowsets(const Version& spec_version, - vector* rowsets) const { - vector version_path; + std::vector* rowsets) const { + std::vector version_path; RETURN_NOT_OK(capture_consistent_versions(spec_version, &version_path)); RETURN_NOT_OK(_capture_consistent_rowsets_unlocked(version_path, rowsets)); return OLAP_SUCCESS; } -OLAPStatus Tablet::_capture_consistent_rowsets_unlocked(const vector& version_path, - vector* rowsets) const { +OLAPStatus Tablet::_capture_consistent_rowsets_unlocked(const std::vector& version_path, + std::vector* rowsets) const { DCHECK(rowsets != nullptr && rowsets->empty()); rowsets->reserve(version_path.size()); for (auto& version : version_path) { @@ -654,15 +654,15 @@ OLAPStatus Tablet::_capture_consistent_rowsets_unlocked(const vector& v } OLAPStatus Tablet::capture_rs_readers(const Version& spec_version, - vector* rs_readers) const { - vector version_path; + std::vector* rs_readers) const { + std::vector version_path; RETURN_NOT_OK(capture_consistent_versions(spec_version, &version_path)); RETURN_NOT_OK(capture_rs_readers(version_path, rs_readers)); return OLAP_SUCCESS; } -OLAPStatus Tablet::capture_rs_readers(const vector& version_path, - vector* rs_readers) const { +OLAPStatus Tablet::capture_rs_readers(const std::vector& version_path, + std::vector* rs_readers) const { DCHECK(rs_readers != nullptr && rs_readers->empty()); for (auto version : version_path) { auto it = _rs_version_map.find(version); @@ -710,7 +710,7 @@ AlterTabletTaskSharedPtr Tablet::alter_task() { void Tablet::add_alter_task(int64_t related_tablet_id, int32_t related_schema_hash, - const vector& versions_to_alter, + const std::vector& versions_to_alter, const AlterTabletType alter_type) { AlterTabletTask alter_task; alter_task.set_alter_state(ALTER_RUNNING); @@ -801,7 +801,7 @@ void Tablet::compute_version_hash_from_rowsets( *version_hash = v_hash; } -void Tablet::calc_missed_versions(int64_t spec_version, vector* missed_versions) { +void Tablet::calc_missed_versions(int64_t spec_version, std::vector* missed_versions) { ReadLock rdlock(&_meta_lock); calc_missed_versions_unlocked(spec_version, missed_versions); } @@ -811,7 +811,7 @@ void Tablet::calc_missed_versions(int64_t spec_version, vector* missed_ // [0-4][5-5][8-8][9-9] // if spec_version = 6, we still return {6, 7} other than {7} void Tablet::calc_missed_versions_unlocked(int64_t spec_version, - vector* missed_versions) const { + std::vector* missed_versions) const { DCHECK(spec_version > 0) << "invalid spec_version: " << spec_version; std::list existing_versions; for (auto& rs : _tablet_meta->all_rs_metas()) { @@ -850,7 +850,7 @@ void Tablet::max_continuous_version_from_beginning(Version* version, void Tablet::_max_continuous_version_from_beginning_unlocked(Version* version, VersionHash* v_hash) const { - vector> existing_versions; + std::vector> existing_versions; for (auto& rs : _tablet_meta->all_rs_metas()) { existing_versions.emplace_back(rs->version() , rs->version_hash()); } @@ -892,7 +892,7 @@ void Tablet::calculate_cumulative_point() { OLAPStatus Tablet::split_range(const OlapTuple& start_key_strings, const OlapTuple& end_key_strings, uint64_t request_block_row_count, - vector* ranges) { + std::vector* ranges) { DCHECK(ranges != nullptr); RowCursor start_key; diff --git a/be/src/olap/tablet_manager.cpp b/be/src/olap/tablet_manager.cpp index 5cdfe7bc4c..0f6ef104fc 100644 --- a/be/src/olap/tablet_manager.cpp +++ b/be/src/olap/tablet_manager.cpp @@ -175,7 +175,7 @@ OLAPStatus TabletManager::_add_tablet_to_map_unlocked(TTabletId tablet_id, Schem if (drop_old) { // If the new tablet is fresher than the existing one, then replace // the existing tablet with the new one. - RETURN_NOT_OK_LOG(_drop_tablet_unlocked(tablet_id, schema_hash, keep_files), Substitute( + RETURN_NOT_OK_LOG(_drop_tablet_unlocked(tablet_id, schema_hash, keep_files), strings::Substitute( "failed to drop old tablet when add new tablet. tablet_id=$0, schema_hash=$1", tablet_id, schema_hash)); } @@ -332,7 +332,7 @@ TabletSharedPtr TabletManager::_internal_create_tablet_unlocked( } else { // add alter task to new tablet if it is a new tablet during schema change tablet->add_alter_task(base_tablet->tablet_id(), base_tablet->schema_hash(), - vector(), alter_type); + std::vector(), alter_type); } // 有可能出现以下2种特殊情况: // 1. 因为操作系统时间跳变,导致新生成的表的creation_time小于旧表的creation_time时间 @@ -542,7 +542,7 @@ OLAPStatus TabletManager::_drop_tablet_unlocked( } OLAPStatus TabletManager::drop_tablets_on_error_root_path( - const vector& tablet_info_vec) { + const std::vector& tablet_info_vec) { OLAPStatus res = OLAP_SUCCESS; for (int32 i = 0; i < _tablet_map_lock_shard_size; i++) { RWMutex& tablet_map_lock = _tablet_map_lock_array[i]; @@ -681,7 +681,7 @@ void TabletManager::get_tablet_stat(TTabletStatResult* result) { TabletSharedPtr TabletManager::find_best_tablet_to_compaction( CompactionType compaction_type, DataDir* data_dir, - vector &tablet_submitted_compaction) { + std::vector &tablet_submitted_compaction) { int64_t now_ms = UnixMillis(); const string& compaction_type_str = compaction_type == CompactionType::BASE_COMPACTION ? "base" : "cumulative"; double highest_score = 0.0; @@ -693,7 +693,7 @@ TabletSharedPtr TabletManager::find_best_tablet_to_compaction( tablet_map_t& tablet_map = _tablet_map_array[i]; for (tablet_map_t::value_type& table_ins : tablet_map){ for (TabletSharedPtr& tablet_ptr : table_ins.second.table_arr) { - vector::iterator it_tablet = + std::vector::iterator it_tablet = find(tablet_submitted_compaction.begin(), tablet_submitted_compaction.end(), tablet_ptr->tablet_id()); if (it_tablet != tablet_submitted_compaction.end()) { @@ -850,10 +850,10 @@ OLAPStatus TabletManager::load_tablet_from_meta(DataDir* data_dir, TTabletId tab return OLAP_ERR_TABLE_INDEX_VALIDATE_ERROR; } - RETURN_NOT_OK_LOG(tablet->init(), Substitute("tablet init failed. tablet=$0", + RETURN_NOT_OK_LOG(tablet->init(), strings::Substitute("tablet init failed. tablet=$0", tablet->full_name())); RETURN_NOT_OK_LOG(_add_tablet_unlocked(tablet_id, schema_hash, tablet, update_meta, force), - Substitute("fail to add tablet. tablet=$0", tablet->full_name())); + strings::Substitute("fail to add tablet. tablet=$0", tablet->full_name())); return OLAP_SUCCESS; } @@ -877,7 +877,7 @@ OLAPStatus TabletManager::load_tablet_from_dir(DataDir* store, TTabletId tablet_ int32_t shard = stol(shard_str); // load dir is called by clone, restore, storage migration // should change tablet uid when tablet object changed - RETURN_NOT_OK_LOG(TabletMeta::reset_tablet_uid(header_path), Substitute( + RETURN_NOT_OK_LOG(TabletMeta::reset_tablet_uid(header_path), strings::Substitute( "failed to set tablet uid when copied meta file. header_path=%0", header_path));; if (!Env::Default()->path_exists(header_path).ok()) { @@ -897,7 +897,7 @@ OLAPStatus TabletManager::load_tablet_from_dir(DataDir* store, TTabletId tablet_ tablet_meta->serialize(&meta_binary); RETURN_NOT_OK_LOG(load_tablet_from_meta(store, tablet_id, schema_hash, meta_binary, true, force, restore), - Substitute("fail to load tablet. header_path=$0", header_path)); + strings::Substitute("fail to load tablet. header_path=$0", header_path)); return OLAP_SUCCESS; } @@ -1200,7 +1200,7 @@ void TabletManager::get_partition_related_tablets(int64_t partition_id, } void TabletManager::do_tablet_meta_checkpoint(DataDir* data_dir) { - vector related_tablets; + std::vector related_tablets; { for (int32 i = 0 ; i < _tablet_map_lock_shard_size; i++) { ReadLock tablet_map_rdlock(&_tablet_map_lock_array[i]); diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp index 5169cfb7f8..b014eaff0c 100755 --- a/be/src/olap/tablet_meta.cpp +++ b/be/src/olap/tablet_meta.cpp @@ -236,7 +236,7 @@ OLAPStatus TabletMeta::reset_tablet_uid(const string& header_file) { return res; } -string TabletMeta::construct_header_file_path(const string& schema_hash_path, +std::string TabletMeta::construct_header_file_path(const string& schema_hash_path, int64_t tablet_id) { std::stringstream header_name_stream; header_name_stream << schema_hash_path << "/" << tablet_id << ".hdr"; @@ -495,7 +495,7 @@ OLAPStatus TabletMeta::add_rs_meta(const RowsetMetaSharedPtr& rs_meta) { } void TabletMeta::delete_rs_meta_by_version(const Version& version, - vector* deleted_rs_metas) { + std::vector* deleted_rs_metas) { auto it = _rs_metas.begin(); while (it != _rs_metas.end()) { if ((*it)->version() == version) { @@ -510,8 +510,8 @@ void TabletMeta::delete_rs_meta_by_version(const Version& version, } } -void TabletMeta::modify_rs_metas(const vector& to_add, - const vector& to_delete) { +void TabletMeta::modify_rs_metas(const std::vector& to_add, + const std::vector& to_delete) { // Remove to_delete rowsets from _rs_metas for (auto rs_to_del : to_delete) { auto it = _rs_metas.begin(); @@ -691,7 +691,7 @@ OLAPStatus TabletMeta::set_alter_state(AlterTabletState alter_state) { } } -string TabletMeta::full_name() const { +std::string TabletMeta::full_name() const { std::stringstream ss; ss << _tablet_id << "." << _schema_hash diff --git a/be/src/olap/task/engine_alter_tablet_task.cpp b/be/src/olap/task/engine_alter_tablet_task.cpp index 0e7e1172e4..0baef9c246 100644 --- a/be/src/olap/task/engine_alter_tablet_task.cpp +++ b/be/src/olap/task/engine_alter_tablet_task.cpp @@ -24,7 +24,7 @@ namespace doris { using std::to_string; EngineAlterTabletTask::EngineAlterTabletTask(const TAlterTabletReqV2& request, - int64_t signature, const TTaskType::type task_type, vector* error_msgs, + int64_t signature, const TTaskType::type task_type, std::vector* error_msgs, const string& process_name): _alter_tablet_req(request), _signature(signature), diff --git a/be/src/olap/task/engine_batch_load_task.cpp b/be/src/olap/task/engine_batch_load_task.cpp index cf9c5a83cc..5b9820fbb0 100644 --- a/be/src/olap/task/engine_batch_load_task.cpp +++ b/be/src/olap/task/engine_batch_load_task.cpp @@ -286,7 +286,7 @@ AgentStatus EngineBatchLoadTask::_process() { } OLAPStatus EngineBatchLoadTask::_push(const TPushReq& request, - vector* tablet_info_vec) { + std::vector* tablet_info_vec) { OLAPStatus res = OLAP_SUCCESS; LOG(INFO) << "begin to process push. " << " transaction_id=" << request.transaction_id @@ -350,7 +350,7 @@ OLAPStatus EngineBatchLoadTask::_push(const TPushReq& request, OLAPStatus EngineBatchLoadTask::_delete_data( const TPushReq& request, - vector* tablet_info_vec) { + std::vector* tablet_info_vec) { LOG(INFO) << "begin to process delete data. request=" << ThriftDebugString(request); DorisMetrics::instance()->delete_requests_total->increment(1); diff --git a/be/src/olap/task/engine_clone_task.cpp b/be/src/olap/task/engine_clone_task.cpp index 22d1f82fd6..264d45c828 100644 --- a/be/src/olap/task/engine_clone_task.cpp +++ b/be/src/olap/task/engine_clone_task.cpp @@ -52,8 +52,8 @@ const uint32_t GET_LENGTH_TIMEOUT = 10; EngineCloneTask::EngineCloneTask(const TCloneReq& clone_req, const TMasterInfo& master_info, int64_t signature, - vector* error_msgs, - vector* tablet_infos, + std::vector* error_msgs, + std::vector* tablet_infos, AgentStatus* res_status) : _clone_req(clone_req), _error_msgs(error_msgs), @@ -100,7 +100,7 @@ OLAPStatus EngineCloneTask::_do_clone() { RowsetSharedPtr clone_rowset = tablet->get_rowset_by_version(clone_version); if (clone_rowset == nullptr) { // try to incremental clone - vector missed_versions; + std::vector missed_versions; tablet->calc_missed_versions(_clone_req.committed_version, &missed_versions); LOG(INFO) << "finish to calculate missed versions when clone. " << "tablet=" << tablet->full_name() @@ -169,7 +169,7 @@ OLAPStatus EngineCloneTask::_do_clone() { _error_msgs->push_back("clone get local root path failed."); status = DORIS_ERROR; } - stringstream tablet_dir_stream; + std::stringstream tablet_dir_stream; tablet_dir_stream << local_shard_root_path << "/" << _clone_req.tablet_id << "/" << _clone_req.schema_hash; @@ -187,7 +187,7 @@ OLAPStatus EngineCloneTask::_do_clone() { if (status == DORIS_SUCCESS) { LOG(INFO) << "clone copy done. src_host: " << src_host.host << " src_file_path: " << src_file_path; - stringstream schema_hash_path_stream; + std::stringstream schema_hash_path_stream; schema_hash_path_stream << local_shard_root_path << "/" << _clone_req.tablet_id << "/" << _clone_req.schema_hash; @@ -215,7 +215,7 @@ OLAPStatus EngineCloneTask::_do_clone() { } // Clean useless dir, if failed, ignore it. if (status != DORIS_SUCCESS && status != DORIS_CREATE_TABLE_EXIST) { - stringstream local_data_path_stream; + std::stringstream local_data_path_stream; local_data_path_stream << local_shard_root_path << "/" << _clone_req.tablet_id; string local_data_path = local_data_path_stream.str(); @@ -295,8 +295,8 @@ AgentStatus EngineCloneTask::_clone_copy( const string& local_data_path, TBackend* src_host, string* snapshot_path, - vector* error_msgs, - const vector* missed_versions, + std::vector* error_msgs, + const std::vector* missed_versions, bool* allow_incremental_clone) { AgentStatus status = DORIS_SUCCESS; @@ -485,7 +485,7 @@ Status EngineCloneTask::_download_files( return Status::OK(); }; RETURN_IF_ERROR(HttpClient::execute_with_retry(DOWNLOAD_FILE_MAX_RETRY, 1, list_files_cb)); - vector file_name_list = strings::Split(file_list_str, "\n", strings::SkipWhitespace()); + std::vector file_name_list = strings::Split(file_list_str, "\n", strings::SkipWhitespace()); // If the header file is not exist, the table couldn't loaded by olap engine. // Avoid of data is not complete, we copy the header file at last. @@ -609,14 +609,14 @@ OLAPStatus EngineCloneTask::_convert_to_new_snapshot(const string& clone_dir, in } OlapSnapshotConverter converter; TabletMetaPB tablet_meta_pb; - vector pending_rowsets; + std::vector pending_rowsets; res = converter.to_new_snapshot(olap_header_msg, clone_dir, clone_dir, &tablet_meta_pb, &pending_rowsets, false); if (res != OLAP_SUCCESS) { LOG(WARNING) << "fail to convert snapshot to new format. dir='" << clone_dir; return res; } - vector files_to_delete; + std::vector files_to_delete; for (auto file_name : clone_files) { string full_file_path = clone_dir + "/" + file_name; files_to_delete.push_back(full_file_path); @@ -638,7 +638,7 @@ OLAPStatus EngineCloneTask::_convert_to_new_snapshot(const string& clone_dir, in OLAPStatus EngineCloneTask::_finish_clone(Tablet* tablet, const string& clone_dir, int64_t committed_version, bool is_incremental_clone) { OLAPStatus res = OLAP_SUCCESS; - vector linked_success_files; + std::vector linked_success_files; // clone and compaction operation should be performed sequentially tablet->obtain_base_compaction_lock(); @@ -749,11 +749,11 @@ OLAPStatus EngineCloneTask::_clone_incremental_data(Tablet* tablet, const Tablet LOG(INFO) << "begin to incremental clone. tablet=" << tablet->full_name() << ", committed_version=" << committed_version; - vector missed_versions; + std::vector missed_versions; tablet->calc_missed_versions_unlocked(committed_version, &missed_versions); - vector versions_to_delete; - vector rowsets_to_clone; + std::vector versions_to_delete; + std::vector rowsets_to_clone; VLOG(3) << "get missed versions again when finish incremental clone. " << "tablet=" << tablet->full_name() @@ -783,8 +783,8 @@ OLAPStatus EngineCloneTask::_clone_full_data(Tablet* tablet, TabletMeta* cloned_ LOG(INFO) << "begin to full clone. tablet=" << tablet->full_name() << ", cloned_max_version=" << cloned_max_version.first << "-" << cloned_max_version.second; - vector versions_to_delete; - vector rs_metas_found_in_src; + std::vector versions_to_delete; + std::vector rs_metas_found_in_src; // check local versions for (auto& rs_meta : tablet->tablet_meta()->all_rs_metas()) { Version local_version(rs_meta->start_version(), rs_meta->end_version()); @@ -835,7 +835,7 @@ OLAPStatus EngineCloneTask::_clone_full_data(Tablet* tablet, TabletMeta* cloned_ } } } - vector rowsets_to_clone; + std::vector rowsets_to_clone; for (auto& rs_meta : cloned_tablet_meta->all_rs_metas()) { rowsets_to_clone.push_back(rs_meta); LOG(INFO) << "Delta to clone." diff --git a/be/src/olap/task/engine_publish_version_task.cpp b/be/src/olap/task/engine_publish_version_task.cpp index 28202ab9cb..7fb1ce2ca0 100644 --- a/be/src/olap/task/engine_publish_version_task.cpp +++ b/be/src/olap/task/engine_publish_version_task.cpp @@ -26,7 +26,7 @@ namespace doris { using std::map; EnginePublishVersionTask::EnginePublishVersionTask(TPublishVersionRequest& publish_version_req, - vector* error_tablet_ids) : + std::vector* error_tablet_ids) : _publish_version_req(publish_version_req), _error_tablet_ids(error_tablet_ids) {} diff --git a/be/src/olap/task/engine_storage_migration_task.cpp b/be/src/olap/task/engine_storage_migration_task.cpp index 7a47324928..44938d0d60 100644 --- a/be/src/olap/task/engine_storage_migration_task.cpp +++ b/be/src/olap/task/engine_storage_migration_task.cpp @@ -77,7 +77,7 @@ OLAPStatus EngineStorageMigrationTask::_migrate() { } int32_t end_version = last_version->end_version(); - vector consistent_rowsets; + std::vector consistent_rowsets; res = _tablet->capture_consistent_rowsets(Version(0, end_version), &consistent_rowsets); if (consistent_rowsets.empty()) { _tablet->release_header_lock(); @@ -94,7 +94,7 @@ OLAPStatus EngineStorageMigrationTask::_migrate() { LOG(WARNING) << "fail to get shard from store: " << _dest_store->path(); break; } - stringstream root_path_stream; + std::stringstream root_path_stream; root_path_stream << _dest_store->path() << DATA_PREFIX << "/" << shard; string full_path = SnapshotManager::instance()->get_schema_hash_full_path(_tablet, root_path_stream.str()); // if dir already exist then return err, it should not happen. @@ -189,7 +189,7 @@ void EngineStorageMigrationTask::_generate_new_header( TabletMetaSharedPtr new_tablet_meta) { _tablet->generate_tablet_meta_copy_unlocked(new_tablet_meta); - vector rs_metas; + std::vector rs_metas; for (auto& rs : consistent_rowsets) { rs_metas.push_back(rs->rowset_meta()); } diff --git a/be/src/olap/utils.cpp b/be/src/olap/utils.cpp index f34108f288..255e02f151 100644 --- a/be/src/olap/utils.cpp +++ b/be/src/olap/utils.cpp @@ -56,7 +56,7 @@ using std::string; using std::set; using std::vector; -using std::unique_ptr; + namespace doris { @@ -1014,12 +1014,12 @@ OLAPStatus read_write_test_file(const string& test_file_path) { LOG(WARNING) << "fail to allocate write buffer memory. size=" << TEST_FILE_BUF_SIZE; return OLAP_ERR_MALLOC_ERROR; } - unique_ptr write_buff (write_test_buff, &std::free); + std::unique_ptr write_buff (write_test_buff, &std::free); if (posix_memalign((void**) &read_test_buff, DIRECT_IO_ALIGNMENT, TEST_FILE_BUF_SIZE)!= 0) { LOG(WARNING) << "fail to allocate read buffer memory. size=" << TEST_FILE_BUF_SIZE; return OLAP_ERR_MALLOC_ERROR; } - unique_ptr read_buff (read_test_buff, &std::free); + std::unique_ptr read_buff (read_test_buff, &std::free); // generate random numbers uint32_t rand_seed = static_cast(time(NULL)); for (size_t i = 0; i < TEST_FILE_BUF_SIZE; ++i) { diff --git a/be/src/plugin/plugin_loader.cpp b/be/src/plugin/plugin_loader.cpp index 32c0d2abdb..4f87b20dcb 100644 --- a/be/src/plugin/plugin_loader.cpp +++ b/be/src/plugin/plugin_loader.cpp @@ -45,7 +45,7 @@ Status PluginLoader::open_valid() { Status PluginLoader::close_valid() { if (_plugin.get() != nullptr && (_plugin->flags & PLUGIN_NOT_DYNAMIC_UNINSTALL)) { - return Status::InternalError(Substitute("plugin $0 not allow dynamic uninstall", _name)); + return Status::InternalError(strings::Substitute("plugin $0 not allow dynamic uninstall", _name)); } return Status::OK(); diff --git a/be/src/plugin/plugin_mgr.cpp b/be/src/plugin/plugin_mgr.cpp index 78379035db..d7b41bbd4f 100644 --- a/be/src/plugin/plugin_mgr.cpp +++ b/be/src/plugin/plugin_mgr.cpp @@ -27,7 +27,7 @@ using namespace strings; #define PLUGIN_TYPE_CHECK(_type) { \ if (_type >= PLUGIN_TYPE_MAX) { \ - return Status::InvalidArgument(Substitute("error plugin type: $0", _type)); \ + return Status::InvalidArgument(strings::Substitute("error plugin type: $0", _type)); \ } \ } @@ -92,7 +92,7 @@ Status PluginMgr::get_plugin(const std::string& name, int type, std::shared_ptr< return Status::OK(); } - return Status::NotFound(Substitute("not found type $0 plugin $1", type, name)); + return Status::NotFound(strings::Substitute("not found type $0 plugin $1", type, name)); } Status PluginMgr::get_plugin(const std::string& name, std::shared_ptr* plugin) { @@ -107,7 +107,7 @@ Status PluginMgr::get_plugin(const std::string& name, std::shared_ptr* p } } - return Status::NotFound(Substitute("not found plugin $0", name)); + return Status::NotFound(strings::Substitute("not found plugin $0", name)); } Status PluginMgr::get_plugin_list(int type, std::vector>* plugin_list) { @@ -129,7 +129,7 @@ Status PluginMgr::register_builtin_plugin(const std::string& name, int type, con auto iter = _plugins[type].find(name); if (iter != _plugins[type].end()) { - return Status::AlreadyExist(Substitute("the type $0 plugin $1 already register", type, name)); + return Status::AlreadyExist(strings::Substitute("the type $0 plugin $1 already register", type, name)); } std::unique_ptr loader = std::unique_ptr(new BuiltinPluginLoader(name, type, plugin)); diff --git a/be/src/plugin/plugin_zip.cpp b/be/src/plugin/plugin_zip.cpp index 599e4e57f4..937aab57ff 100644 --- a/be/src/plugin/plugin_zip.cpp +++ b/be/src/plugin/plugin_zip.cpp @@ -51,10 +51,10 @@ PluginZip::~PluginZip() { Status PluginZip::extract(const std::string& target_dir, const std::string& plugin_name) { // check plugin install path - std::string plugin_install_path = Substitute("$0/$1", target_dir, plugin_name); + std::string plugin_install_path = strings::Substitute("$0/$1", target_dir, plugin_name); if (FileUtils::check_exist(plugin_install_path)) { - return Status::AlreadyExist(Substitute("plugin $0 already install!", plugin_name)); + return Status::AlreadyExist(strings::Substitute("plugin $0 already install!", plugin_name)); } if (!FileUtils::check_exist(target_dir)) { @@ -63,7 +63,7 @@ Status PluginZip::extract(const std::string& target_dir, const std::string& plug std::string zip_path = _source; if (!is_local_source(_source)) { - zip_path = Substitute("$0/.temp_$1_$2.zip", target_dir, GetCurrentTimeMicros(), plugin_name); + zip_path = strings::Substitute("$0/.temp_$1_$2.zip", target_dir, GetCurrentTimeMicros(), plugin_name); _clean_paths.push_back(zip_path); RETURN_IF_ERROR(download(zip_path)); @@ -122,7 +122,7 @@ Status PluginZip::download(const std::string& zip_path) { digest.digest(); if (0 != strncmp(digest.hex().c_str(), expect.c_str(), 32)) { return Status::InternalError( - Substitute("plugin install checksum failed. expect: $0, actual:$1", expect, digest.hex())); + strings::Substitute("plugin install checksum failed. expect: $0, actual:$1", expect, digest.hex())); } return Status::OK(); diff --git a/be/src/runtime/buffered_tuple_stream3.cc b/be/src/runtime/buffered_tuple_stream3.cc index bddbbc4895..43434bc056 100644 --- a/be/src/runtime/buffered_tuple_stream3.cc +++ b/be/src/runtime/buffered_tuple_stream3.cc @@ -34,7 +34,7 @@ #include "util/pretty_printer.h" #include "util/runtime_profile.h" -#include "common/names.h" + #ifdef NDEBUG #define CHECK_CONSISTENCY_FAST() @@ -51,7 +51,7 @@ using BufferHandle = BufferPool::BufferHandle; BufferedTupleStream3::BufferedTupleStream3(RuntimeState* state, const RowDescriptor* row_desc, BufferPool::ClientHandle* buffer_pool_client, - int64_t default_page_len, int64_t max_page_len, const set& ext_varlen_slots) + int64_t default_page_len, int64_t max_page_len, const std::set& ext_varlen_slots) : state_(state), desc_(row_desc), node_id_(-1), @@ -169,7 +169,7 @@ void BufferedTupleStream3::CheckPageConsistency(const Page* page) const { } string BufferedTupleStream3::DebugString() const { - stringstream ss; + std::stringstream ss; ss << "BufferedTupleStream3 num_rows=" << num_rows_ << " rows_returned=" << rows_returned_ << " pinned=" << pinned_ << " delete_on_read=" << delete_on_read_ << " closed=" << closed_ << "\n" @@ -259,7 +259,7 @@ void BufferedTupleStream3::Close(RowBatch* batch, RowBatch::FlushMode flush) { BufferPool::BufferHandle buffer; Status status = buffer_pool_->ExtractBuffer(buffer_pool_client_, &page.handle, &buffer); DCHECK(status.ok()); - batch->add_buffer(buffer_pool_client_, move(buffer), flush); + batch->add_buffer(buffer_pool_client_, std::move(buffer), flush); } else { buffer_pool_->DestroyPage(buffer_pool_client_, &page.handle); } @@ -393,7 +393,7 @@ Status BufferedTupleStream3::CalcPageLenForRow(int64_t row_size, int64_t* page_l // (state_->query_options().max_row_size, TUnit::BYTES); return Status::InternalError(ss.str()); } - *page_len = max(default_page_len_, BitUtil::RoundUpToPowerOfTwo(row_size)); + *page_len = std::max(default_page_len_, BitUtil::RoundUpToPowerOfTwo(row_size)); return Status::OK(); } @@ -695,7 +695,7 @@ void BufferedTupleStream3::UnpinStream(UnpinMode mode) { } */ Status BufferedTupleStream3::GetRows( - const std::shared_ptr& tracker, scoped_ptr* batch, bool* got_rows) { + const std::shared_ptr& tracker, boost::scoped_ptr* batch, bool* got_rows) { if (num_rows() > numeric_limits::max()) { // RowBatch::num_rows_ is a 32-bit int, avoid an overflow. return Status::InternalError(Substitute("Trying to read $0 rows into in-memory batch failed. Limit " diff --git a/be/src/runtime/bufferpool/buffer_allocator.cc b/be/src/runtime/bufferpool/buffer_allocator.cc index 22fe25500c..e5d0131c40 100644 --- a/be/src/runtime/bufferpool/buffer_allocator.cc +++ b/be/src/runtime/bufferpool/buffer_allocator.cc @@ -28,7 +28,6 @@ #include "util/pretty_printer.h" #include "util/runtime_profile.h" -#include "common/names.h" #include "common/config.h" //DECLARE_bool(disable_mem_pools); @@ -72,7 +71,7 @@ class BufferPool::FreeBufferArena : public CacheLineAligned { /// /// Caller should not hold 'lock_'. If 'arena_lock' is non-null, ownership of the /// arena lock is transferred to the caller. Uses std::unique_lock instead of - /// boost::unique_lock because it is movable. + /// boost::std::unique_lock because it is movable. std::pair FreeSystemMemory(int64_t target_bytes_to_free, int64_t target_bytes_to_claim, std::unique_lock* arena_lock); @@ -106,7 +105,7 @@ class BufferPool::FreeBufferArena : public CacheLineAligned { /// it doesn't acquire the arena lock. int64_t GetNumCleanPages(); - string DebugString(); + std::string DebugString(); private: /// The data structures for each power-of-two size of buffers/pages. @@ -119,7 +118,7 @@ class BufferPool::FreeBufferArena : public CacheLineAligned { void AddFreeBuffer(BufferHandle&& buffer) { DCHECK_EQ(num_free_buffers.load(), free_buffers.Size()); num_free_buffers.add(1); - free_buffers.AddFreeBuffer(move(buffer)); + free_buffers.AddFreeBuffer(std::move(buffer)); } /// The number of entries in 'free_buffers'. Can be read without holding a lock to @@ -178,8 +177,8 @@ int64_t BufferPool::BufferAllocator::CalcMaxBufferLen( // Find largest power of 2 smaller than 'system_bytes_limit'. int64_t upper_bound = system_bytes_limit == 0 ? 1L : 1L << BitUtil::Log2Floor64(system_bytes_limit); - upper_bound = min(MAX_BUFFER_BYTES, upper_bound); - return max(min_buffer_len, upper_bound); // Can't be < min_buffer_len. + upper_bound = std::min(MAX_BUFFER_BYTES, upper_bound); + return std::max(min_buffer_len, upper_bound); // Can't be < min_buffer_len. } BufferPool::BufferAllocator::BufferAllocator( @@ -202,9 +201,9 @@ BufferPool::BufferAllocator::BufferAllocator( DCHECK_LE(0, min_buffer_len_); DCHECK_LE(min_buffer_len_, max_buffer_len_); DCHECK_LE(max_buffer_len_, MAX_BUFFER_BYTES); - DCHECK_LE(max_buffer_len_, max(system_bytes_limit_, min_buffer_len_)); + DCHECK_LE(max_buffer_len_, std::max(system_bytes_limit_, min_buffer_len_)); - for (unique_ptr& arena : per_core_arenas_) { + for (std::unique_ptr& arena : per_core_arenas_) { arena.reset(new FreeBufferArena(this)); } } @@ -254,7 +253,7 @@ Status BufferPool::BufferAllocator::AllocateInternal(int64_t len, BufferHandle* int64_t delta = DecreaseBytesRemaining(len, true, &system_bytes_remaining_); if (delta != len) { DCHECK_EQ(0, delta); - const vector& numa_node_cores = CpuInfo::get_cores_of_same_numa_node(current_core); + const std::vector& numa_node_cores = CpuInfo::get_cores_of_same_numa_node(current_core); const int numa_node_core_idx = CpuInfo::get_numa_node_core_idx(current_core); // Fast-ish path: find a buffer of the right size from another core on the same @@ -314,7 +313,7 @@ int64_t DecreaseBytesRemaining( while (true) { int64_t old_value = bytes_remaining->load(); if (require_full_decrease && old_value < max_decrease) return 0; - int64_t decrease = min(old_value, max_decrease); + int64_t decrease = std::min(old_value, max_decrease); int64_t new_value = old_value - decrease; if (bytes_remaining->compare_and_swap(old_value, new_value)) { return decrease; @@ -347,7 +346,7 @@ int64_t BufferPool::BufferAllocator::ScavengeBuffers( // therefore must start at 0 to respect the lock order. Otherwise we start with the // current core's arena for locality and to avoid excessive contention on arena 0. int start_core = slow_but_sure ? 0 : current_core; - vector> arena_locks; + std::vector> arena_locks; if (slow_but_sure) arena_locks.resize(per_core_arenas_.size()); for (int i = 0; i < per_core_arenas_.size(); ++i) { @@ -378,11 +377,11 @@ void BufferPool::BufferAllocator::Free(BufferHandle&& handle) { handle.client_ = nullptr; // Buffer is no longer associated with a client. FreeBufferArena* arena = per_core_arenas_[handle.home_core_].get(); handle.Poison(); - arena->AddFreeBuffer(move(handle)); + arena->AddFreeBuffer(std::move(handle)); } void BufferPool::BufferAllocator::AddCleanPage( - const unique_lock& client_lock, Page* page) { + const std::unique_lock& client_lock, Page* page) { page->client->DCheckHoldsLock(client_lock); FreeBufferArena* arena = per_core_arenas_[page->buffer.home_core_].get(); arena->AddCleanPage(page); @@ -390,11 +389,11 @@ void BufferPool::BufferAllocator::AddCleanPage( bool BufferPool::BufferAllocator::RemoveCleanPage( - const unique_lock& client_lock, bool claim_buffer, Page* page) { + const std::unique_lock& client_lock, bool claim_buffer, Page* page) { page->client->DCheckHoldsLock(client_lock); FreeBufferArena* arena; { - lock_guard pl(page->buffer_lock); + std::lock_guard pl(page->buffer_lock); // Page may be evicted - in which case it has no home core and is not in an arena. if (!page->buffer.is_open()) return false; arena = per_core_arenas_[page->buffer.home_core_].get(); @@ -403,7 +402,7 @@ bool BufferPool::BufferAllocator::RemoveCleanPage( } void BufferPool::BufferAllocator::Maintenance() { - for (unique_ptr& arena : per_core_arenas_) arena->Maintenance(); + for (std::unique_ptr& arena : per_core_arenas_) arena->Maintenance(); } void BufferPool::BufferAllocator::ReleaseMemory(int64_t bytes_to_free) { @@ -422,13 +421,13 @@ int BufferPool::BufferAllocator::GetFreeListSize(int core, int64_t len) { return per_core_arenas_[core]->GetFreeListSize(len); } -int64_t BufferPool::BufferAllocator::FreeToSystem(vector&& buffers) { +int64_t BufferPool::BufferAllocator::FreeToSystem(std::vector&& buffers) { int64_t bytes_freed = 0; for (BufferHandle& buffer : buffers) { bytes_freed += buffer.len(); // Ensure that the memory is unpoisoned when it's next allocated by the system. buffer.Unpoison(); - system_allocator_->Free(move(buffer)); + system_allocator_->Free(std::move(buffer)); } return bytes_freed; } @@ -436,7 +435,7 @@ int64_t BufferPool::BufferAllocator::FreeToSystem(vector&& buffers int64_t BufferPool::BufferAllocator::SumOverArenas( std::function compute_fn) const { int64_t total = 0; - for (const unique_ptr& arena : per_core_arenas_) { + for (const std::unique_ptr& arena : per_core_arenas_) { total += compute_fn(arena.get()); } return total; @@ -463,8 +462,8 @@ int64_t BufferPool::BufferAllocator::GetCleanPageBytes() const { return clean_page_bytes_limit_ - clean_page_bytes_remaining_.load(); } -string BufferPool::BufferAllocator::DebugString() { - stringstream ss; +std::string BufferPool::BufferAllocator::DebugString() { + std::stringstream ss; ss << " " << this << " min_buffer_len: " << min_buffer_len_ << " system_bytes_limit: " << system_bytes_limit_ << " system_bytes_remaining: " << system_bytes_remaining_.load() << "\n" @@ -482,8 +481,8 @@ BufferPool::FreeBufferArena::~FreeBufferArena() { for (int i = 0; i < NumBufferSizes(); ++i) { // Clear out the free lists. FreeList* list = &buffer_sizes_[i].free_buffers; - vector buffers = list->GetBuffersToFree(list->Size()); - parent_->system_bytes_remaining_.add(parent_->FreeToSystem(move(buffers))); + std::vector buffers = list->GetBuffersToFree(list->Size()); + parent_->system_bytes_remaining_.add(parent_->FreeToSystem(std::move(buffers))); // All pages should have been destroyed. DCHECK_EQ(0, buffer_sizes_[i].clean_pages.size()); @@ -491,19 +490,19 @@ BufferPool::FreeBufferArena::~FreeBufferArena() { } void BufferPool::FreeBufferArena::AddFreeBuffer(BufferHandle&& buffer) { - lock_guard al(lock_); + std::lock_guard al(lock_); if (config::disable_mem_pools) { int64_t len = buffer.len(); - parent_->system_allocator_->Free(move(buffer)); + parent_->system_allocator_->Free(std::move(buffer)); parent_->system_bytes_remaining_.add(len); return; } PerSizeLists* lists = GetListsForSize(buffer.len()); - lists->AddFreeBuffer(move(buffer)); + lists->AddFreeBuffer(std::move(buffer)); } bool BufferPool::FreeBufferArena::RemoveCleanPage(bool claim_buffer, Page* page) { - lock_guard al(lock_); + std::lock_guard al(lock_); PerSizeLists* lists = GetListsForSize(page->len); DCHECK_EQ(lists->num_clean_pages.load(), lists->clean_pages.size()); if (!lists->clean_pages.remove(page)) return false; @@ -512,10 +511,10 @@ bool BufferPool::FreeBufferArena::RemoveCleanPage(bool claim_buffer, Page* page) if (!claim_buffer) { BufferHandle buffer; { - lock_guard pl(page->buffer_lock); - buffer = move(page->buffer); + std::lock_guard pl(page->buffer_lock); + buffer = std::move(page->buffer); } - lists->AddFreeBuffer(move(buffer)); + lists->AddFreeBuffer(std::move(buffer)); } return true; } @@ -526,13 +525,13 @@ bool BufferPool::FreeBufferArena::PopFreeBuffer( // Check before acquiring lock. if (lists->num_free_buffers.load() == 0) return false; - lock_guard al(lock_); + std::lock_guard al(lock_); FreeList* list = &lists->free_buffers; DCHECK_EQ(lists->num_free_buffers.load(), list->Size()); if (!list->PopFreeBuffer(buffer)) return false; buffer->Unpoison(); lists->num_free_buffers.add(-1); - lists->low_water_mark = min(lists->low_water_mark, list->Size()); + lists->low_water_mark = std::min(lists->low_water_mark, list->Size()); return true; } /* @@ -542,14 +541,14 @@ bool BufferPool::FreeBufferArena::EvictCleanPage( // Check before acquiring lock. if (lists->num_clean_pages.Load() == 0) return false; - lock_guard al(lock_); + std::lock_guard al(lock_); DCHECK_EQ(lists->num_clean_pages.Load(), lists->clean_pages.size()); Page* page = lists->clean_pages.dequeue(); if (page == nullptr) return false; lists->num_clean_pages.Add(-1); parent_->clean_page_bytes_remaining_.Add(buffer_len); - lock_guard pl(page->buffer_lock); - *buffer = move(page->buffer); + std::lock_guard pl(page->buffer_lock); + *buffer = std::move(page->buffer); return true; } */ @@ -565,7 +564,7 @@ std::pair BufferPool::FreeBufferArena::FreeSystemMemory( std::unique_lock al(lock_, std::defer_lock_t()); if (arena_lock != nullptr) al.lock(); - vector buffers; + std::vector buffers; // Search from largest to smallest to avoid freeing many small buffers unless // necessary. for (int i = NumBufferSizes() - 1; i >= 0; --i) { @@ -584,7 +583,7 @@ std::pair BufferPool::FreeBufferArena::FreeSystemMemory( // Figure out how many of the buffers in the free list we should free. DCHECK_GT(target_bytes_to_free, bytes_freed); const int64_t buffer_len = 1L << (i + parent_->log_min_buffer_len_); - int64_t buffers_to_free = min(free_buffers->Size(), + int64_t buffers_to_free = std::min(free_buffers->Size(), BitUtil::Ceil(target_bytes_to_free - bytes_freed, buffer_len)); int64_t buffer_bytes_to_free = buffers_to_free * buffer_len; @@ -598,14 +597,14 @@ std::pair BufferPool::FreeBufferArena::FreeSystemMemory( if (page == nullptr) break; BufferHandle page_buffer; { - lock_guard pl(page->buffer_lock); - page_buffer = move(page->buffer); + std::lock_guard pl(page->buffer_lock); + page_buffer = std::move(page->buffer); } ++buffers_to_free; buffer_bytes_to_free += page_buffer.len(); ++num_pages_evicted; page_bytes_evicted += page_buffer.len(); - free_buffers->AddFreeBuffer(move(page_buffer)); + free_buffers->AddFreeBuffer(std::move(page_buffer)); } lists->num_free_buffers.add(num_pages_evicted); lists->num_clean_pages.add(-num_pages_evicted); @@ -617,20 +616,20 @@ std::pair BufferPool::FreeBufferArena::FreeSystemMemory( DCHECK_EQ(buffer_bytes_to_free, buffer_bytes_freed); bytes_freed += buffer_bytes_to_free; lists->num_free_buffers.add(-buffers_to_free); - lists->low_water_mark = min(lists->low_water_mark, free_buffers->Size()); + lists->low_water_mark = std::min(lists->low_water_mark, free_buffers->Size()); if (bytes_freed >= target_bytes_to_free) break; } // Should have cleared out all lists if we don't have enough memory at this point. DCHECK_EQ(0, free_buffers->Size()); DCHECK_EQ(0, clean_pages->size()); } - int64_t bytes_claimed = min(bytes_freed, target_bytes_to_claim); + int64_t bytes_claimed = std::min(bytes_freed, target_bytes_to_claim); if (bytes_freed > bytes_claimed) { // Add back the extra for other threads before releasing the lock to avoid race // where the other thread may not be able to find enough buffers. parent_->system_bytes_remaining_.add(bytes_freed - bytes_claimed); } - if (arena_lock != nullptr) *arena_lock = move(al); + if (arena_lock != nullptr) *arena_lock = std::move(al); return std::make_pair(bytes_freed, bytes_claimed); } @@ -638,23 +637,23 @@ void BufferPool::FreeBufferArena::AddCleanPage(Page* page) { bool eviction_needed = config::disable_mem_pools || DecreaseBytesRemaining( page->len, true, &parent_->clean_page_bytes_remaining_) == 0; - lock_guard al(lock_); + std::lock_guard al(lock_); PerSizeLists* lists = GetListsForSize(page->len); DCHECK_EQ(lists->num_clean_pages.load(), lists->clean_pages.size()); if (eviction_needed) { if (lists->clean_pages.empty()) { // No other pages to evict, must evict 'page' instead of adding it. - lists->AddFreeBuffer(move(page->buffer)); + lists->AddFreeBuffer(std::move(page->buffer)); } else { // Evict an older page (FIFO eviction) to make space for this one. Page* page_to_evict = lists->clean_pages.dequeue(); lists->clean_pages.enqueue(page); BufferHandle page_to_evict_buffer; { - lock_guard pl(page_to_evict->buffer_lock); - page_to_evict_buffer = move(page_to_evict->buffer); + std::lock_guard pl(page_to_evict->buffer_lock); + page_to_evict_buffer = std::move(page_to_evict->buffer); } - lists->AddFreeBuffer(move(page_to_evict_buffer)); + lists->AddFreeBuffer(std::move(page_to_evict_buffer)); } } else { lists->clean_pages.enqueue(page); @@ -663,7 +662,7 @@ void BufferPool::FreeBufferArena::AddCleanPage(Page* page) { } void BufferPool::FreeBufferArena::Maintenance() { - lock_guard al(lock_); + std::lock_guard al(lock_); for (int i = 0; i < NumBufferSizes(); ++i) { PerSizeLists* lists = &buffer_sizes_[i]; DCHECK_LE(lists->low_water_mark, lists->free_buffers.Size()); @@ -671,7 +670,7 @@ void BufferPool::FreeBufferArena::Maintenance() { // We haven't needed the buffers below the low water mark since the previous // Maintenance() call. Discard half of them to free up memory. By always discarding // at least one, we guarantee that an idle list will shrink to zero entries. - int num_to_free = max(1, lists->low_water_mark / 2); + int num_to_free = std::max(1, lists->low_water_mark / 2); parent_->system_bytes_remaining_.add( parent_->FreeToSystem(lists->free_buffers.GetBuffersToFree(num_to_free))); lists->num_free_buffers.add(-num_to_free); @@ -681,7 +680,7 @@ void BufferPool::FreeBufferArena::Maintenance() { } int BufferPool::FreeBufferArena::GetFreeListSize(int64_t len) { - lock_guard al(lock_); + std::lock_guard al(lock_); PerSizeLists* lists = GetListsForSize(len); DCHECK_EQ(lists->num_free_buffers.load(), lists->free_buffers.Size()); return lists->free_buffers.Size(); @@ -715,9 +714,9 @@ int64_t BufferPool::FreeBufferArena::GetNumCleanPages() { }); } -string BufferPool::FreeBufferArena::DebugString() { - lock_guard al(lock_); - stringstream ss; +std::string BufferPool::FreeBufferArena::DebugString() { + std::lock_guard al(lock_); + std::stringstream ss; ss << " " << this << "\n"; for (int i = 0; i < NumBufferSizes(); ++i) { int64_t buffer_len = 1L << (parent_->log_min_buffer_len_ + i); @@ -726,7 +725,7 @@ string BufferPool::FreeBufferArena::DebugString() { << " free buffers: " << lists.num_free_buffers.load() << " low water mark: " << lists.low_water_mark << " clean pages: " << lists.num_clean_pages.load() << " "; - lists.clean_pages.iterate(bind(Page::DebugStringCallback, &ss, _1)); + lists.clean_pages.iterate(boost::bind(Page::DebugStringCallback, &ss, _1)); ss << "\n"; } diff --git a/be/src/runtime/bufferpool/buffer_allocator.h b/be/src/runtime/bufferpool/buffer_allocator.h index 5a8b4bf7e4..185f6c482a 100644 --- a/be/src/runtime/bufferpool/buffer_allocator.h +++ b/be/src/runtime/bufferpool/buffer_allocator.h @@ -102,7 +102,7 @@ struct BufferPool::BufferAllocator { /// client's lock via 'client_lock' so that moving the page between the client list and /// the free page list is atomic. Caller must not hold 'FreeBufferArena::lock_' or any /// Page::lock. - void AddCleanPage(const boost::unique_lock& client_lock, Page* page); + void AddCleanPage(const std::unique_lock& client_lock, Page* page); /// Removes a clean page 'page' from a clean page list and returns true, if present in /// one of the lists. Returns true if it was present. If 'claim_buffer' is true, the @@ -112,7 +112,7 @@ struct BufferPool::BufferAllocator { /// client list and the free page list is atomic. Caller must not hold /// 'FreeBufferArena::lock_' or any Page::lock. bool RemoveCleanPage( - const boost::unique_lock& client_lock, bool claim_buffer, Page* page); + const std::unique_lock& client_lock, bool claim_buffer, Page* page); /// Periodically called to release free buffers back to the SystemAllocator. Releases /// buffers based on recent allocation patterns, trying to minimise the number of diff --git a/be/src/runtime/bufferpool/buffer_pool.cc b/be/src/runtime/bufferpool/buffer_pool.cc index da640f6ed6..a09fff0e8a 100644 --- a/be/src/runtime/bufferpool/buffer_pool.cc +++ b/be/src/runtime/bufferpool/buffer_pool.cc @@ -21,7 +21,7 @@ #include #include -#include "common/names.h" + #include "gutil/strings/substitute.h" #include "runtime/bufferpool/buffer_allocator.h" #include "util/bit_util.h" @@ -145,7 +145,7 @@ Status BufferPool::CreatePage( BufferHandle new_buffer; // No changes have been made to state yet, so we can cleanly return on error. RETURN_IF_ERROR(AllocateBuffer(client, len, &new_buffer)); - Page* page = client->impl_->CreatePinnedPage(move(new_buffer)); + Page* page = client->impl_->CreatePinnedPage(std::move(new_buffer)); handle->Open(page, client); if (buffer != nullptr) *buffer = &page->buffer; return Status::OK(); @@ -243,7 +243,7 @@ void BufferPool::FreeBuffer(ClientHandle* client, BufferHandle* handle) { if (!handle->is_open()) return; // Should be idempotent. DCHECK_EQ(client, handle->client_); int64_t len = handle->len_; - allocator_->Free(move(*handle)); + allocator_->Free(std::move(*handle)); client->impl_->FreedBuffer(len); } @@ -398,10 +398,10 @@ BufferPool::Client::Client(BufferPool* pool, //TmpFileMgr::FileGroup* file_group BufferPool::Page* BufferPool::Client::CreatePinnedPage(BufferHandle&& buffer) { Page* page = new Page(this, buffer.len()); - page->buffer = move(buffer); + page->buffer = std::move(buffer); page->pin_count = 1; - boost::lock_guard lock(lock_); + std::lock_guard lock(lock_); // The buffer is transferred to the page so will be accounted for in // pinned_pages_.bytes() instead of buffers_allocated_bytes_. buffers_allocated_bytes_ -= page->len; @@ -418,7 +418,7 @@ void BufferPool::Client::DestroyPageInternal( Page* page = handle->page_; // Remove the page from the list that it is currently present in (if any). { - unique_lock cl(lock_); + std::unique_lock cl(lock_); // First try to remove from the pinned or dirty unpinned lists. if (!pinned_pages_.remove(page) && !dirty_unpinned_pages_.remove(page)) { // The page either has a write in flight, is clean, or is evicted. @@ -441,7 +441,7 @@ void BufferPool::Client::DestroyPageInternal( *out_buffer = std::move(page->buffer); buffers_allocated_bytes_ += out_buffer->len(); } else if (page->buffer.is_open()) { - pool_->allocator_->Free(move(page->buffer)); + pool_->allocator_->Free(std::move(page->buffer)); } delete page; handle->Reset(); @@ -453,7 +453,7 @@ void BufferPool::Client::MoveToDirtyUnpinned(Page* page) { // DCHECK(spilling_enabled()); DCHECK_EQ(0, page->pin_count); - unique_lock lock(lock_); + std::unique_lock lock(lock_); DCHECK_CONSISTENCY(); DCHECK(pinned_pages_.contains(page)); pinned_pages_.remove(page); @@ -464,7 +464,7 @@ void BufferPool::Client::MoveToDirtyUnpinned(Page* page) { } Status BufferPool::Client::StartMoveToPinned(ClientHandle* client, Page* page) { - unique_lock cl(lock_); + std::unique_lock cl(lock_); DCHECK_CONSISTENCY(); // Propagate any write errors that occurred for this client. //RETURN_IF_ERROR(write_status_i; @@ -503,7 +503,7 @@ Status BufferPool::Client::StartMoveToPinned(ClientHandle* client, Page* page) { } /* Status BufferPool::Client::StartMoveEvictedToPinned( - unique_lock* client_lock, ClientHandle* client, Page* page) { + unique_lock* client_lock, ClientHandle* client, Page* page) { DCHECK(!page->buffer.is_open()); // Safe to modify the page's buffer handle without holding the page lock because no @@ -529,7 +529,7 @@ void BufferPool::Client::UndoMoveEvictedToPinned(Page* page) { page->write_handle->CancelRead(); page->pin_in_flight = false; - unique_lock lock(lock_); + unique_lock lock(lock_); DCHECK_CONSISTENCY(); DCHECK(pinned_pages_.contains(page)); pinned_pages_.remove(page); @@ -553,7 +553,7 @@ Status BufferPool::Client::FinishMoveEvictedToPinned(Page* page) { } */ Status BufferPool::Client::PrepareToAllocateBuffer(int64_t len) { - unique_lock lock(lock_); + std::unique_lock lock(lock_); // Clean enough pages to allow allocation to proceed without violating our eviction // policy. This can fail, so only update the accounting once success is ensured. //RETURN_IF_ERROR(CleanPages(&lock, len)); @@ -564,11 +564,11 @@ Status BufferPool::Client::PrepareToAllocateBuffer(int64_t len) { } Status BufferPool::Client::DecreaseReservationTo(int64_t target_bytes) { - unique_lock lock(lock_); + std::unique_lock lock(lock_); int64_t current_reservation = reservation_.GetReservation(); DCHECK_GE(current_reservation, target_bytes); int64_t amount_to_free = - min(reservation_.GetUnusedReservation(), current_reservation - target_bytes); + std::min(reservation_.GetUnusedReservation(), current_reservation - target_bytes); if (amount_to_free == 0) return Status::OK(); // Clean enough pages to allow us to safely release reservation. //RETURN_IF_ERROR(CleanPages(&lock, amount_to_free)); @@ -576,7 +576,7 @@ Status BufferPool::Client::DecreaseReservationTo(int64_t target_bytes) { return Status::OK(); } -Status BufferPool::Client::CleanPages(unique_lock* client_lock, int64_t len) { +Status BufferPool::Client::CleanPages(std::unique_lock* client_lock, int64_t len) { DCheckHoldsLock(*client_lock); DCHECK_CONSISTENCY(); /* @@ -635,7 +635,7 @@ void BufferPool::Client::WriteDirtyPagesAsync(int64_t min_bytes_to_write) { Page* page = dirty_unpinned_pages_.tail(); // LIFO. DCHECK(page != NULL) << "Should have been enough dirty unpinned pages"; { - lock_guard pl(page->buffer_lock); + std::lock_guard pl(page->buffer_lock); DCHECK(file_group_ != NULL); DCHECK(page->buffer.is_open()); COUNTER_ADD(counters().bytes_written, page->len); @@ -665,7 +665,7 @@ void BufferPool::Client::WriteCompleteCallback(Page* page, const Status& write_s if (debug_write_delay_ms_ > 0) SleepForMs(debug_write_delay_ms_); #endif { - unique_lock cl(lock_); + std::unique_lock cl(lock_); DCHECK(in_flight_write_pages_.contains(page)); // The status should always be propagated. // TODO: if we add cancellation support to TmpFileMgr, consider cancellation path. @@ -683,7 +683,7 @@ void BufferPool::Client::WriteCompleteCallback(Page* page, const Status& write_s } } -void BufferPool::Client::WaitForWrite(unique_lock* client_lock, Page* page) { +void BufferPool::Client::WaitForWrite(std::unique_lock* client_lock, Page* page) { DCheckHoldsLock(*client_lock); while (in_flight_write_pages_.contains(page)) { SCOPED_TIMER(counters().write_wait_time); @@ -692,26 +692,26 @@ void BufferPool::Client::WaitForWrite(unique_lock* client_lock, Page* pag } void BufferPool::Client::WaitForAllWrites() { - unique_lock cl(lock_); + std::unique_lock cl(lock_); while (in_flight_write_pages_.size() > 0) { write_complete_cv_.Wait(cl); } } */ string BufferPool::Client::DebugString() { - lock_guard lock(lock_); - stringstream ss; + std::lock_guard lock(lock_); + std::stringstream ss; ss << " " << this << " name: " << name_ << " write_status: " << write_status_.get_error_msg() << " buffers allocated " << buffers_allocated_bytes_ << " num_pages: " << num_pages_ << " pinned_bytes: " << pinned_pages_.bytes() << " dirty_unpinned_bytes: " << dirty_unpinned_pages_.bytes() << " in_flight_write_bytes: " << in_flight_write_pages_.bytes() << " reservation: " << reservation_.DebugString(); ss << "\n " << pinned_pages_.size() << " pinned pages: "; - pinned_pages_.iterate(bind(Page::DebugStringCallback, &ss, _1)); + pinned_pages_.iterate(boost::bind(Page::DebugStringCallback, &ss, _1)); ss << "\n " << dirty_unpinned_pages_.size() << " dirty unpinned pages: "; - dirty_unpinned_pages_.iterate(bind(Page::DebugStringCallback, &ss, _1)); + dirty_unpinned_pages_.iterate(boost::bind(Page::DebugStringCallback, &ss, _1)); ss << "\n " << in_flight_write_pages_.size() << " in flight write pages: "; - in_flight_write_pages_.iterate(bind(Page::DebugStringCallback, &ss, _1)); + in_flight_write_pages_.iterate(boost::bind(Page::DebugStringCallback, &ss, _1)); return ss.str(); } @@ -729,7 +729,7 @@ string BufferPool::ClientHandle::DebugString() const { /* string BufferPool::PageHandle::DebugString() const { if (is_open()) { - lock_guard pl(page_->buffer_lock); + std::lock_guard pl(page_->buffer_lock); return Substitute(" $0 client: $1/$2 page: {$3}", this, client_, client_->impl_, page_->DebugString()); } else { @@ -744,8 +744,8 @@ string BufferPool::Page::DebugString() { return ss.str(); } -bool BufferPool::Page::DebugStringCallback(stringstream* ss, BufferPool::Page* page) { - lock_guard pl(page->buffer_lock); +bool BufferPool::Page::DebugStringCallback(std::stringstream* ss, BufferPool::Page* page) { + std::lock_guard pl(page->buffer_lock); (*ss) << page->DebugString() << "\n"; return true; } @@ -762,7 +762,7 @@ string BufferPool::BufferHandle::DebugString() const { } string BufferPool::DebugString() { - stringstream ss; + std::stringstream ss; ss << " " << this << " min_buffer_len: " << min_buffer_len_ << "\n" << allocator_->DebugString(); return ss.str(); diff --git a/be/src/runtime/bufferpool/buffer_pool_internal.h b/be/src/runtime/bufferpool/buffer_pool_internal.h index 1ad3b02e01..01df1becef 100644 --- a/be/src/runtime/bufferpool/buffer_pool_internal.h +++ b/be/src/runtime/bufferpool/buffer_pool_internal.h @@ -21,7 +21,7 @@ #include #include -#include +#include #include "runtime/bufferpool/buffer_pool_counters.h" #include "runtime/bufferpool/buffer_pool.h" @@ -194,7 +194,7 @@ class BufferPool::Client { /// internal accounting and release the buffer to the client's reservation. No page or /// client locks should be held by the caller. void FreedBuffer(int64_t len) { - boost::lock_guard cl(lock_); + std::lock_guard cl(lock_); reservation_.ReleaseTo(len); buffers_allocated_bytes_ -= len; DCHECK_CONSISTENCY(); @@ -203,14 +203,14 @@ class BufferPool::Client { /// Wait for the in-flight write for 'page' to complete. /// 'lock_' must be held by the caller via 'client_lock'. page->buffer_lock should /// not be held. - //void WaitForWrite(boost::unique_lock* client_lock, Page* page); + //void WaitForWrite(boost::unique_lock* client_lock, Page* page); /// Test helper: wait for all in-flight writes to complete. /// 'lock_' must not be held by the caller. //void WaitForAllWrites(); /// Asserts that 'client_lock' is holding 'lock_'. - void DCheckHoldsLock(const boost::unique_lock& client_lock) { + void DCheckHoldsLock(const std::unique_lock& client_lock) { DCHECK(client_lock.mutex() == &lock_ && client_lock.owns_lock()); } @@ -246,7 +246,7 @@ class BufferPool::Client { /// enough dirty pages are flushed to disk to satisfy the buffer pool's internal /// invariants after the allocation. 'lock_' should be held by the caller via /// 'client_lock' - Status CleanPages(boost::unique_lock* client_lock, int64_t len); + Status CleanPages(std::unique_lock* client_lock, int64_t len); /// Initiates asynchronous writes of dirty unpinned pages to disk. Ensures that at /// least 'min_bytes_to_write' bytes of writes will be written asynchronously. May @@ -263,7 +263,7 @@ class BufferPool::Client { /// locked by the caller via 'client_lock' and handle->page must be unlocked. /// 'client_lock' is released then reacquired. //Status StartMoveEvictedToPinned( - // boost::unique_lock* client_lock, ClientHandle* client, Page* page); + // std::unique_lock* client_lock, ClientHandle* client, Page* page); /// The buffer pool that owns the client. BufferPool* const pool_; @@ -287,7 +287,7 @@ class BufferPool::Client { int debug_write_delay_ms_; /// Lock to protect the below member variables; - boost::mutex lock_; + std::mutex lock_; /// All non-OK statuses returned by write operations are merged into this status. /// All operations that depend on pages being written to disk successfully (e.g. diff --git a/be/src/runtime/bufferpool/reservation_tracker.cc b/be/src/runtime/bufferpool/reservation_tracker.cc index 41620f3157..fad7ab6c69 100644 --- a/be/src/runtime/bufferpool/reservation_tracker.cc +++ b/be/src/runtime/bufferpool/reservation_tracker.cc @@ -26,7 +26,7 @@ #include "util/dummy_runtime_profile.h" #include "util/runtime_profile.h" -#include "common/names.h" + #include "olap/utils.h" namespace doris { @@ -39,7 +39,7 @@ ReservationTracker::~ReservationTracker() { void ReservationTracker::InitRootTracker( RuntimeProfile* profile, int64_t reservation_limit) { - lock_guard l(lock_); + std::lock_guard l(lock_); DCHECK(!initialized_); parent_ = nullptr; mem_tracker_ = nullptr; @@ -60,7 +60,7 @@ void ReservationTracker::InitChildTracker(RuntimeProfile* profile, DCHECK(parent != nullptr); DCHECK_GE(reservation_limit, 0); - lock_guard l(lock_); + std::lock_guard l(lock_); DCHECK(!initialized_); parent_ = parent; mem_tracker_ = mem_tracker; @@ -118,7 +118,7 @@ void ReservationTracker::InitCounters( } void ReservationTracker::Close() { - lock_guard l(lock_); + std::lock_guard l(lock_); if (!initialized_) return; CheckConsistency(); DCHECK_EQ(used_reservation_, 0); @@ -131,12 +131,12 @@ void ReservationTracker::Close() { } bool ReservationTracker::IncreaseReservation(int64_t bytes) { - lock_guard l(lock_); + std::lock_guard l(lock_); return IncreaseReservationInternalLocked(bytes, false, false); } bool ReservationTracker::IncreaseReservationToFit(int64_t bytes) { - lock_guard l(lock_); + std::lock_guard l(lock_); return IncreaseReservationInternalLocked(bytes, true, false); } @@ -144,7 +144,7 @@ bool ReservationTracker::IncreaseReservationInternalLocked( int64_t bytes, bool use_existing_reservation, bool is_child_reservation) { DCHECK(initialized_); int64_t reservation_increase = - use_existing_reservation ? max(0, bytes - unused_reservation()) : bytes; + use_existing_reservation ? std::max(0, bytes - unused_reservation()) : bytes; DCHECK_GE(reservation_increase, 0); bool granted; @@ -163,7 +163,7 @@ bool ReservationTracker::IncreaseReservationInternalLocked( if (parent_ == nullptr) { granted = true; } else { - lock_guard l(parent_->lock_); + std::lock_guard l(parent_->lock_); granted = parent_->IncreaseReservationInternalLocked(reservation_increase, true, true); } @@ -212,7 +212,7 @@ void ReservationTracker::ReleaseToMemTracker(int64_t reservation_decrease) { } void ReservationTracker::DecreaseReservation(int64_t bytes, bool is_child_reservation) { - lock_guard l(lock_); + std::lock_guard l(lock_); DecreaseReservationLocked(bytes, is_child_reservation); } @@ -232,8 +232,8 @@ void ReservationTracker::DecreaseReservationLocked( bool ReservationTracker::TransferReservationTo(ReservationTracker* other, int64_t bytes) { if (other == this) return true; // Find the path to the root from both. The root is guaranteed to be a common ancestor. - vector path_to_common = FindPathToRoot(); - vector other_path_to_common = other->FindPathToRoot(); + std::vector path_to_common = FindPathToRoot(); + std::vector other_path_to_common = other->FindPathToRoot(); DCHECK_EQ(path_to_common.back(), other_path_to_common.back()); ReservationTracker* common_ancestor = path_to_common.back(); // Remove any common ancestors - they do not need to be updated for this transfer. @@ -259,7 +259,7 @@ bool ReservationTracker::TransferReservationTo(ReservationTracker* other, int64_ // Lock all of the trackers so we can do the update atomically. Need to be careful to // lock subtrees in the correct order. - vector> locks; + std::vector> locks; bool lock_first = path_to_common.empty() || other_path_to_common.empty() || lock_sibling_subtree_first(path_to_common.back(), other_path_to_common.back()); if (lock_first) { @@ -298,21 +298,21 @@ bool ReservationTracker::TransferReservationTo(ReservationTracker* other, int64_ // Update the 'child_reservations_' on the common ancestor if needed. // Case 1: reservation was pushed up to 'other'. if (common_ancestor == other) { - lock_guard l(other->lock_); + std::lock_guard l(other->lock_); other->child_reservations_ -= bytes; other->CheckConsistency(); } // Case 2: reservation was pushed down below 'this'. if (common_ancestor == this) { - lock_guard l(lock_); + std::lock_guard l(lock_); child_reservations_ += bytes; CheckConsistency(); } return true; } -vector ReservationTracker::FindPathToRoot() { - vector path_to_root; +std::vector ReservationTracker::FindPathToRoot() { + std::vector path_to_root; ReservationTracker* curr = this; do { path_to_root.push_back(curr); @@ -322,7 +322,7 @@ vector ReservationTracker::FindPathToRoot() { } void ReservationTracker::AllocateFrom(int64_t bytes) { - lock_guard l(lock_); + std::lock_guard l(lock_); DCHECK(initialized_); DCHECK_GE(bytes, 0); DCHECK_LE(bytes, unused_reservation()); @@ -331,7 +331,7 @@ void ReservationTracker::AllocateFrom(int64_t bytes) { } void ReservationTracker::ReleaseTo(int64_t bytes) { - lock_guard l(lock_); + std::lock_guard l(lock_); DCHECK(initialized_); DCHECK_GE(bytes, 0); DCHECK_LE(bytes, used_reservation_); @@ -340,25 +340,25 @@ void ReservationTracker::ReleaseTo(int64_t bytes) { } int64_t ReservationTracker::GetReservation() { - lock_guard l(lock_); + std::lock_guard l(lock_); DCHECK(initialized_); return reservation_; } int64_t ReservationTracker::GetUsedReservation() { - lock_guard l(lock_); + std::lock_guard l(lock_); DCHECK(initialized_); return used_reservation_; } int64_t ReservationTracker::GetUnusedReservation() { - lock_guard l(lock_); + std::lock_guard l(lock_); DCHECK(initialized_); return unused_reservation(); } int64_t ReservationTracker::GetChildReservations() { - lock_guard l(lock_); + std::lock_guard l(lock_); DCHECK(initialized_); return child_reservations_; } @@ -397,11 +397,11 @@ void ReservationTracker::UpdateReservation(int64_t delta) { CheckConsistency(); } -string ReservationTracker::DebugString() { - //lock_guard l(lock_); +std::string ReservationTracker::DebugString() { + //std::lock_guard l(lock_); if (!initialized_) return ": uninitialized"; - string parent_debug_string = parent_ == nullptr ? "NULL" : parent_->DebugString(); + std::string parent_debug_string = parent_ == nullptr ? "NULL" : parent_->DebugString(); std::stringstream ss; ss << ": reservation_limit " << reservation_limit_ << " reservation " << reservation_ << " used_reservation " << used_reservation_ diff --git a/be/src/runtime/bufferpool/suballocator.cc b/be/src/runtime/bufferpool/suballocator.cc index 80e6ae29d6..a6fc1aee1d 100644 --- a/be/src/runtime/bufferpool/suballocator.cc +++ b/be/src/runtime/bufferpool/suballocator.cc @@ -22,7 +22,7 @@ #include "runtime/bufferpool/reservation_tracker.h" #include "util/bit_util.h" -#include "common/names.h" + #include "gutil/strings/substitute.h" namespace doris { @@ -45,17 +45,17 @@ Suballocator::~Suballocator() { } } -Status Suballocator::Allocate(int64_t bytes, unique_ptr* result) { +Status Suballocator::Allocate(int64_t bytes, std::unique_ptr* result) { DCHECK_GE(bytes, 0); if (UNLIKELY(bytes > MAX_ALLOCATION_BYTES)) { std::stringstream err_stream; err_stream << "Requested memory allocation of " << bytes - << " bytes, larger than max " << "supported of " << MAX_ALLOCATION_BYTES + << " bytes, larger than std::max " << "supported of " << MAX_ALLOCATION_BYTES << " bytes"; return Status::InternalError(err_stream.str()); } - unique_ptr free_node; - bytes = max(bytes, MIN_ALLOCATION_BYTES); + std::unique_ptr free_node; + bytes = std::max(bytes, MIN_ALLOCATION_BYTES); const int target_list_idx = ComputeListIndex(bytes); for (int i = target_list_idx; i < NUM_FREE_LISTS; ++i) { free_node = PopFreeListHead(i); @@ -74,13 +74,13 @@ Status Suballocator::Allocate(int64_t bytes, unique_ptr* result) // Free node may be larger than required. const int free_list_idx = ComputeListIndex(free_node->len_); if (free_list_idx != target_list_idx) { - RETURN_IF_ERROR(SplitToSize(move(free_node), bytes, &free_node)); + RETURN_IF_ERROR(SplitToSize(std::move(free_node), bytes, &free_node)); DCHECK(free_node != nullptr); } free_node->in_use_ = true; allocated_ += free_node->len_; - *result = move(free_node); + *result = std::move(free_node); return Status::OK(); } @@ -89,34 +89,34 @@ int Suballocator::ComputeListIndex(int64_t bytes) const { } uint64_t Suballocator::ComputeAllocateBufferSize(int64_t bytes) const { - bytes = max(bytes, MIN_ALLOCATION_BYTES); + bytes = std::max(bytes, MIN_ALLOCATION_BYTES); const int target_list_idx = ComputeListIndex(bytes); for (int i = target_list_idx; i < NUM_FREE_LISTS; ++i) { if (CheckFreeListHeadNotNull(i)) return 0; } - return max(min_buffer_len_, BitUtil::RoundUpToPowerOfTwo(bytes)); + return std::max(min_buffer_len_, BitUtil::RoundUpToPowerOfTwo(bytes)); } -Status Suballocator::AllocateBuffer(int64_t bytes, unique_ptr* result) { +Status Suballocator::AllocateBuffer(int64_t bytes, std::unique_ptr* result) { DCHECK_LE(bytes, MAX_ALLOCATION_BYTES); - const int64_t buffer_len = max(min_buffer_len_, BitUtil::RoundUpToPowerOfTwo(bytes)); + const int64_t buffer_len = std::max(min_buffer_len_, BitUtil::RoundUpToPowerOfTwo(bytes)); if (!client_->IncreaseReservationToFit(buffer_len)) { *result = nullptr; return Status::OK(); } - unique_ptr free_node; + std::unique_ptr free_node; RETURN_IF_ERROR(Suballocation::Create(&free_node)); RETURN_IF_ERROR(pool_->AllocateBuffer(client_, buffer_len, &free_node->buffer_)); free_node->data_ = free_node->buffer_.data(); free_node->len_ = buffer_len; - *result = move(free_node); + *result = std::move(free_node); return Status::OK(); } -Status Suballocator::SplitToSize(unique_ptr free_node, - int64_t target_bytes, unique_ptr* result) { +Status Suballocator::SplitToSize(std::unique_ptr free_node, + int64_t target_bytes, std::unique_ptr* result) { DCHECK(!free_node->in_use_); DCHECK_GT(free_node->len_, target_bytes); @@ -127,12 +127,12 @@ Status Suballocator::SplitToSize(unique_ptr free_node, // Need two nodes per level for the left and right children. const int num_nodes = (free_list_idx - target_list_idx) * 2; constexpr int MAX_NUM_NODES = NUM_FREE_LISTS * 2; - unique_ptr nodes[MAX_NUM_NODES]; + std::unique_ptr nodes[MAX_NUM_NODES]; for (int i = 0; i < num_nodes; ++i) { if (!Suballocation::Create(&nodes[i]).ok()) { // Add the free node to the free list to restore the allocator to an internally // consistent state. - AddToFreeList(move(free_node)); + AddToFreeList(std::move(free_node)); return Status::InternalError("Failed to allocate list node in Suballocator"); } } @@ -142,8 +142,8 @@ Status Suballocator::SplitToSize(unique_ptr free_node, int next_node = 0; for (int i = free_list_idx; i > target_list_idx; --i) { DCHECK_EQ(free_node->len_, 1LL << (i + LOG_MIN_ALLOCATION_BYTES)); - unique_ptr left_child = move(nodes[next_node++]); - unique_ptr right_child = move(nodes[next_node++]); + std::unique_ptr left_child = std::move(nodes[next_node++]); + std::unique_ptr right_child = std::move(nodes[next_node++]); DCHECK_LE(next_node, num_nodes); const int64_t child_len = free_node->len_ / 2; @@ -153,16 +153,16 @@ Status Suballocator::SplitToSize(unique_ptr free_node, left_child->buddy_ = right_child.get(); right_child->buddy_ = left_child.get(); free_node->in_use_ = true; - left_child->parent_ = move(free_node); + left_child->parent_ = std::move(free_node); - AddToFreeList(move(right_child)); - free_node = move(left_child); + AddToFreeList(std::move(right_child)); + free_node = std::move(left_child); } - *result = move(free_node); + *result = std::move(free_node); return Status::OK(); } -uint64_t Suballocator::Free(unique_ptr allocation) { +uint64_t Suballocator::Free(std::unique_ptr allocation) { if (allocation == nullptr) return 0; DCHECK(allocation->in_use_); @@ -172,15 +172,15 @@ uint64_t Suballocator::Free(unique_ptr allocation) { // Iteratively coalesce buddies until the buddy is in use or we get to the root. // This ensures that all buddies in the free lists are coalesced. I.e. we do not // have two buddies in the same free list. - unique_ptr curr_allocation = move(allocation); + std::unique_ptr curr_allocation = std::move(allocation); while (curr_allocation->buddy_ != nullptr) { if (curr_allocation->buddy_->in_use_) { // If the buddy is not free we can't coalesce, just add it to free list. - AddToFreeList(move(curr_allocation)); + AddToFreeList(std::move(curr_allocation)); return 0; } - unique_ptr buddy = RemoveFromFreeList(curr_allocation->buddy_); - curr_allocation = CoalesceBuddies(move(curr_allocation), move(buddy)); + std::unique_ptr buddy = RemoveFromFreeList(curr_allocation->buddy_); + curr_allocation = CoalesceBuddies(std::move(curr_allocation), std::move(buddy)); } // Reached root, which is an entire free buffer. We are not using it, so free up memory. @@ -191,18 +191,18 @@ uint64_t Suballocator::Free(unique_ptr allocation) { return free_len; } -void Suballocator::AddToFreeList(unique_ptr node) { +void Suballocator::AddToFreeList(std::unique_ptr node) { DCHECK(!node->in_use_); int list_idx = ComputeListIndex(node->len_); if (free_lists_[list_idx] != nullptr) { free_lists_[list_idx]->prev_free_ = node.get(); } - node->next_free_ = move(free_lists_[list_idx]); + node->next_free_ = std::move(free_lists_[list_idx]); DCHECK(node->prev_free_ == nullptr); - free_lists_[list_idx] = move(node); + free_lists_[list_idx] = std::move(node); } -unique_ptr Suballocator::RemoveFromFreeList(Suballocation* node) { +std::unique_ptr Suballocator::RemoveFromFreeList(Suballocation* node) { DCHECK(node != nullptr); const int list_idx = ComputeListIndex(node->len_); @@ -210,23 +210,23 @@ unique_ptr Suballocator::RemoveFromFreeList(Suballocation* node) node->next_free_->prev_free_ = node->prev_free_; } - unique_ptr* ptr_from_prev = node->prev_free_ == nullptr ? + std::unique_ptr* ptr_from_prev = node->prev_free_ == nullptr ? &free_lists_[list_idx] : &node->prev_free_->next_free_; node->prev_free_ = nullptr; - unique_ptr result = move(*ptr_from_prev); - *ptr_from_prev = move(node->next_free_); + std::unique_ptr result = std::move(*ptr_from_prev); + *ptr_from_prev = std::move(node->next_free_); return result; } -unique_ptr Suballocator::PopFreeListHead(int list_idx) { +std::unique_ptr Suballocator::PopFreeListHead(int list_idx) { if (free_lists_[list_idx] == nullptr) return nullptr; - unique_ptr result = move(free_lists_[list_idx]); + std::unique_ptr result = std::move(free_lists_[list_idx]); DCHECK(result->prev_free_ == nullptr); if (result->next_free_ != nullptr) { result->next_free_->prev_free_ = nullptr; } - free_lists_[list_idx] = move(result->next_free_); + free_lists_[list_idx] = std::move(result->next_free_); return result; } @@ -234,25 +234,25 @@ bool Suballocator::CheckFreeListHeadNotNull(int list_idx) const { return free_lists_[list_idx] != nullptr; } -unique_ptr Suballocator::CoalesceBuddies( - unique_ptr b1, unique_ptr b2) { +std::unique_ptr Suballocator::CoalesceBuddies( + std::unique_ptr b1, std::unique_ptr b2) { DCHECK(!b1->in_use_); DCHECK(!b2->in_use_); DCHECK_EQ(b1->buddy_, b2.get()); DCHECK_EQ(b2->buddy_, b1.get()); // Only the left child's parent should be present. DCHECK((b1->parent_ != nullptr) ^ (b2->parent_ != nullptr)); - unique_ptr parent = - b1->parent_ != nullptr ? move(b1->parent_) : move(b2->parent_); + std::unique_ptr parent = + b1->parent_ != nullptr ? std::move(b1->parent_) : std::move(b2->parent_); parent->in_use_ = false; return parent; } -Status Suballocation::Create(unique_ptr* new_suballocation) { +Status Suballocation::Create(std::unique_ptr* new_suballocation) { // Allocate from system allocator for simplicity. We don't expect this to be // performance critical or to be used for small allocations where CPU/memory // overhead of these allocations might be a consideration. - new_suballocation->reset(new (nothrow) Suballocation()); + new_suballocation->reset(new (std::nothrow) Suballocation()); if (*new_suballocation == nullptr) { return Status::MemoryAllocFailed("allocate memory failed"); } diff --git a/be/src/runtime/bufferpool/system_allocator.cc b/be/src/runtime/bufferpool/system_allocator.cc index 1bbfbb9f77..f59e3d8e6f 100644 --- a/be/src/runtime/bufferpool/system_allocator.cc +++ b/be/src/runtime/bufferpool/system_allocator.cc @@ -24,7 +24,7 @@ #include "gutil/strings/substitute.h" #include "util/bit_util.h" -#include "common/names.h" + #include "common/config.h" #include "util/error_util.h" diff --git a/be/src/runtime/initial_reservations.cc b/be/src/runtime/initial_reservations.cc index bd2d0e1ea4..76d7a23929 100644 --- a/be/src/runtime/initial_reservations.cc +++ b/be/src/runtime/initial_reservations.cc @@ -29,7 +29,7 @@ #include "util/uid_util.h" #include "util/pretty_printer.h" -#include "common/names.h" + using std::numeric_limits; @@ -62,7 +62,7 @@ Status InitialReservations::Init( void InitialReservations::Claim(BufferPool::ClientHandle* dst, int64_t bytes) { DCHECK_GE(bytes, 0); - lock_guard l(lock_); + std::lock_guard l(lock_); DCHECK_LE(bytes, remaining_initial_reservation_claims_); bool success = dst->TransferReservationFrom(&initial_reservations_, bytes); DCHECK(success) << "Planner computation should ensure enough initial reservations"; @@ -70,7 +70,7 @@ void InitialReservations::Claim(BufferPool::ClientHandle* dst, int64_t bytes) { } void InitialReservations::Return(BufferPool::ClientHandle* src, int64_t bytes) { - lock_guard l(lock_); + std::lock_guard l(lock_); bool success = src->TransferReservationTo(&initial_reservations_, bytes); // No limits on our tracker - no way this should fail. DCHECK(success); diff --git a/be/src/runtime/mem_pool.cpp b/be/src/runtime/mem_pool.cpp index 61b70c94df..3faa173d02 100644 --- a/be/src/runtime/mem_pool.cpp +++ b/be/src/runtime/mem_pool.cpp @@ -25,7 +25,7 @@ #include #include -#include "common/names.h" + namespace doris { @@ -114,7 +114,7 @@ bool MemPool::find_chunk(size_t min_size, bool check_limits) { chunk_size = std::max(min_size, alignof(max_align_t)); } else { DCHECK_GE(next_chunk_size_, INITIAL_CHUNK_SIZE); - chunk_size = max(min_size, next_chunk_size_); + chunk_size = std::max(min_size, next_chunk_size_); } chunk_size = BitUtil::RoundUpToPowerOfTwo(chunk_size); @@ -141,7 +141,7 @@ bool MemPool::find_chunk(size_t min_size, bool check_limits) { total_reserved_bytes_ += chunk_size; // Don't increment the chunk size until the allocation succeeds: if an attempted // large allocation fails we don't want to increase the chunk size further. - next_chunk_size_ = static_cast(min(chunk_size * 2, MAX_CHUNK_SIZE)); + next_chunk_size_ = static_cast(std::min(chunk_size * 2, MAX_CHUNK_SIZE)); DCHECK(check_integrity(true)); return true; @@ -217,8 +217,8 @@ void MemPool::exchange_data(MemPool* other) { other->mem_tracker_->Release(delta_size); } -string MemPool::debug_string() { - stringstream out; +std::string MemPool::debug_string() { + std::stringstream out; char str[16]; out << "MemPool(#chunks=" << chunks_.size() << " ["; for (int i = 0; i < chunks_.size(); ++i) { diff --git a/be/src/runtime/mem_tracker.cpp b/be/src/runtime/mem_tracker.cpp index f52befddde..1461328f9e 100644 --- a/be/src/runtime/mem_tracker.cpp +++ b/be/src/runtime/mem_tracker.cpp @@ -46,17 +46,17 @@ using std::pair; using std::priority_queue; using std::shared_ptr; using std::string; -using std::unique_ptr; + using std::vector; using std::weak_ptr; using strings::Substitute; namespace doris { -const string MemTracker::COUNTER_NAME = "PeakMemoryUsage"; +const std::string MemTracker::COUNTER_NAME = "PeakMemoryUsage"; // Name for request pool MemTrackers. '$0' is replaced with the pool name. -const string REQUEST_POOL_MEM_TRACKER_LABEL_FORMAT = "RequestPool=$0"; +const std::string REQUEST_POOL_MEM_TRACKER_LABEL_FORMAT = "RequestPool=$0"; /// Calculate the soft limit for a MemTracker based on the hard limit 'limit'. static int64_t CalcSoftLimit(int64_t limit) { @@ -225,7 +225,7 @@ std::shared_ptr PoolMemTrackerRegistry::GetRequestPoolMemTracker( if (!create_if_not_present) return nullptr; // First time this pool_name registered, make a new object. std::shared_ptr tracker = MemTracker::CreateTracker( - -1, Substitute(REQUEST_POOL_MEM_TRACKER_LABEL_FORMAT, pool_name), + -1, strings::Substitute(REQUEST_POOL_MEM_TRACKER_LABEL_FORMAT, pool_name), ExecEnv::GetInstance()->process_mem_tracker()); tracker->pool_name_ = pool_name; pool_to_mem_trackers_.emplace(pool_name, std::shared_ptr(tracker)); @@ -272,16 +272,16 @@ void MemTracker::ListTrackers(vector>* trackers) { } //void MemTracker::RegisterMetrics(MetricGroup* metrics, const string& prefix) { -// num_gcs_metric_ = metrics->AddCounter(Substitute("$0.num-gcs", prefix), 0); +// num_gcs_metric_ = metrics->AddCounter(strings::Substitute("$0.num-gcs", prefix), 0); // // // TODO: Consider a total amount of bytes freed counter // bytes_freed_by_last_gc_metric_ = metrics->AddGauge( -// Substitute("$0.bytes-freed-by-last-gc", prefix), -1); +// strings::Substitute("$0.bytes-freed-by-last-gc", prefix), -1); // // bytes_over_limit_metric_ = metrics->AddGauge( -// Substitute("$0.bytes-over-limit", prefix), -1); +// strings::Substitute("$0.bytes-over-limit", prefix), -1); // -// limit_metric_ = metrics->AddGauge(Substitute("$0.limit", prefix), limit_); +// limit_metric_ = metrics->AddGauge(strings::Substitute("$0.limit", prefix), limit_); //} void MemTracker::TransferTo(MemTracker* dst, int64_t bytes) { @@ -324,7 +324,7 @@ void MemTracker::TransferTo(MemTracker* dst, int64_t bytes) { // TrackerName: Limit=5.00 MB Reservation=5.00 MB OtherMemory=1.04 MB // Total=6.04 MB Peak=6.45 MB // -string MemTracker::LogUsage(int max_recursive_depth, const string& prefix, +std::string MemTracker::LogUsage(int max_recursive_depth, const string& prefix, int64_t* logged_consumption) { // Make sure the consumption is up to date. if (consumption_metric_ != nullptr) RefreshConsumptionFromMetric(); @@ -334,7 +334,7 @@ string MemTracker::LogUsage(int max_recursive_depth, const string& prefix, if (!log_usage_if_zero_ && curr_consumption == 0) return ""; - stringstream ss; + std::stringstream ss; ss << prefix << label_ << ":"; if (CheckLimitExceeded(MemLimit::HARD)) ss << " memory limit exceeded."; if (limit_ > 0) ss << " Limit=" << PrettyPrinter::print(limit_, TUnit::BYTES); @@ -363,9 +363,9 @@ string MemTracker::LogUsage(int max_recursive_depth, const string& prefix, if (max_recursive_depth == 0) return ss.str(); // Recurse and get information about the children - string new_prefix = Substitute(" $0", prefix); + std::string new_prefix = strings::Substitute(" $0", prefix); int64_t child_consumption; - string child_trackers_usage; + std::string child_trackers_usage; list> children; { lock_guard l(child_trackers_lock_); @@ -387,15 +387,15 @@ string MemTracker::LogUsage(int max_recursive_depth, const string& prefix, return ss.str(); } -string MemTracker::LogUsage(int max_recursive_depth, const string& prefix, +std::string MemTracker::LogUsage(int max_recursive_depth, const string& prefix, const list>& trackers, int64_t* logged_consumption) { *logged_consumption = 0; - vector usage_strings; + std::vector usage_strings; for (const auto& tracker_weak : trackers) { shared_ptr tracker = tracker_weak.lock(); if (tracker) { int64_t tracker_consumption; - string usage_string = tracker->LogUsage(max_recursive_depth, prefix, + std::string usage_string = tracker->LogUsage(max_recursive_depth, prefix, &tracker_consumption); if (!usage_string.empty()) usage_strings.push_back(usage_string); *logged_consumption += tracker_consumption; @@ -404,14 +404,14 @@ string MemTracker::LogUsage(int max_recursive_depth, const string& prefix, return join(usage_strings, "\n"); } -string MemTracker::LogTopNQueries(int limit) { +std::string MemTracker::LogTopNQueries(int limit) { if (limit == 0) return ""; if (this->is_query_mem_tracker_) return LogUsage(0); - priority_queue, vector>, + priority_queue, std::vector>, std::greater>> min_pq; GetTopNQueries(min_pq, limit); - vector usage_strings(min_pq.size()); + std::vector usage_strings(min_pq.size()); while (!min_pq.empty()) { usage_strings.push_back(min_pq.top().second); min_pq.pop(); @@ -421,7 +421,7 @@ string MemTracker::LogTopNQueries(int limit) { } void MemTracker::GetTopNQueries( - priority_queue, vector>, + priority_queue, std::vector>, greater>>& min_pq, int limit) { list> children; @@ -453,22 +453,22 @@ MemTracker* MemTracker::GetQueryMemTracker() { Status MemTracker::MemLimitExceeded(MemTracker* mtracker, RuntimeState* state, const std::string& details, int64_t failed_allocation_size) { DCHECK_GE(failed_allocation_size, 0); - stringstream ss; - if (!details.empty()) ss << details << endl; + std::stringstream ss; + if (!details.empty()) ss << details << std::endl; if (failed_allocation_size != 0) { if (mtracker != nullptr) ss << mtracker->label(); ss << " could not allocate " << PrettyPrinter::print(failed_allocation_size, TUnit::BYTES) - << " without exceeding limit." << endl; + << " without exceeding limit." << std::endl; } ss << "Error occurred on backend " << BackendOptions::get_localhost(); if (state != nullptr) ss << " by fragment " << print_id(state->fragment_instance_id()); - ss << endl; + ss << std::endl; ExecEnv* exec_env = ExecEnv::GetInstance(); MemTracker* process_tracker = exec_env->process_mem_tracker().get(); const int64_t process_capacity = process_tracker->SpareCapacity(MemLimit::HARD); ss << "Memory left in process limit: " - << PrettyPrinter::print(process_capacity, TUnit::BYTES) << endl; + << PrettyPrinter::print(process_capacity, TUnit::BYTES) << std::endl; // Always log the query tracker (if available). MemTracker* query_tracker = nullptr; @@ -479,7 +479,7 @@ Status MemTracker::MemLimitExceeded(MemTracker* mtracker, RuntimeState* state, const int64_t query_capacity = query_tracker->limit() - query_tracker->consumption(); ss << "Memory left in query limit: " - << PrettyPrinter::print(query_capacity, TUnit::BYTES) << endl; + << PrettyPrinter::print(query_capacity, TUnit::BYTES) << std::endl; } ss << query_tracker->LogUsage(UNLIMITED_DEPTH); } diff --git a/be/src/runtime/raw_value.cpp b/be/src/runtime/raw_value.cpp index c0fa6fbf85..eca1a39177 100644 --- a/be/src/runtime/raw_value.cpp +++ b/be/src/runtime/raw_value.cpp @@ -195,7 +195,7 @@ void RawValue::print_value(const void* value, const TypeDescriptor& type, int sc std::string tmp; bool val = false; - // Special case types that we can print more efficiently without using a stringstream + // Special case types that we can print more efficiently without using a std::stringstream switch (type.type) { case TYPE_BOOLEAN: val = *reinterpret_cast(value); diff --git a/be/src/runtime/row_batch.cpp b/be/src/runtime/row_batch.cpp index 2fd4fcf366..9775ff20a6 100644 --- a/be/src/runtime/row_batch.cpp +++ b/be/src/runtime/row_batch.cpp @@ -130,7 +130,7 @@ RowBatch::RowBatch(const RowDescriptor& row_desc, if (!_row_desc.has_varlen_slots()) { return; } - const vector& tuple_descs = _row_desc.tuple_descriptors(); + const std::vector& tuple_descs = _row_desc.tuple_descriptors(); // For every unique tuple, convert string offsets contained in tuple data into // pointers. Tuples were serialized in the order we are deserializing them in, @@ -138,7 +138,7 @@ RowBatch::RowBatch(const RowDescriptor& row_desc, // we already converted. for (int i = 0; i < _num_rows; ++i) { TupleRow* row = get_row(i); - vector::const_iterator desc = tuple_descs.begin(); + std::vector::const_iterator desc = tuple_descs.begin(); for (int j = 0; desc != tuple_descs.end(); ++desc, ++j) { if ((*desc)->string_slots().empty()) { continue; @@ -231,7 +231,7 @@ RowBatch::RowBatch(const RowDescriptor& row_desc, const TRowBatch& input_batch, if (!_row_desc.has_varlen_slots()) { return; } - const vector& tuple_descs = _row_desc.tuple_descriptors(); + const std::vector& tuple_descs = _row_desc.tuple_descriptors(); // For every unique tuple, convert string offsets contained in tuple data into // pointers. Tuples were serialized in the order we are deserializing them in, @@ -239,7 +239,7 @@ RowBatch::RowBatch(const RowDescriptor& row_desc, const TRowBatch& input_batch, // we already converted. for (int i = 0; i < _num_rows; ++i) { TupleRow* row = get_row(i); - vector::const_iterator desc = tuple_descs.begin(); + std::vector::const_iterator desc = tuple_descs.begin(); for (int j = 0; desc != tuple_descs.end(); ++desc, ++j) { if ((*desc)->string_slots().empty()) { continue; @@ -250,7 +250,7 @@ RowBatch::RowBatch(const RowDescriptor& row_desc, const TRowBatch& input_batch, continue; } - vector::const_iterator slot = (*desc)->string_slots().begin(); + std::vector::const_iterator slot = (*desc)->string_slots().begin(); for (; slot != (*desc)->string_slots().end(); ++slot) { DCHECK((*slot)->type().is_string_type()); StringValue* string_val = tuple->get_string_slot((*slot)->tuple_offset()); @@ -321,8 +321,8 @@ int RowBatch::serialize(TRowBatch* output_batch) { for (int i = 0; i < _num_rows; ++i) { TupleRow* row = get_row(i); - const vector& tuple_descs = _row_desc.tuple_descriptors(); - vector::const_iterator desc = tuple_descs.begin(); + const std::vector& tuple_descs = _row_desc.tuple_descriptors(); + std::vector::const_iterator desc = tuple_descs.begin(); for (int j = 0; desc != tuple_descs.end(); ++desc, ++j) { if (row->get_tuple(j) == NULL) { @@ -389,8 +389,8 @@ int RowBatch::serialize(PRowBatch* output_batch) { char* tuple_data = const_cast(mutable_tuple_data->data()); for (int i = 0; i < _num_rows; ++i) { TupleRow* row = get_row(i); - const vector& tuple_descs = _row_desc.tuple_descriptors(); - vector::const_iterator desc = tuple_descs.begin(); + const std::vector& tuple_descs = _row_desc.tuple_descriptors(); + std::vector::const_iterator desc = tuple_descs.begin(); for (int j = 0; desc != tuple_descs.end(); ++desc, ++j) { if (row->get_tuple(j) == nullptr) { // NULLs are encoded as -1 @@ -615,8 +615,8 @@ int RowBatch::total_byte_size() { // Sum total variable length byte sizes. for (int i = 0; i < _num_rows; ++i) { TupleRow* row = get_row(i); - const vector& tuple_descs = _row_desc.tuple_descriptors(); - vector::const_iterator desc = tuple_descs.begin(); + const std::vector& tuple_descs = _row_desc.tuple_descriptors(); + std::vector::const_iterator desc = tuple_descs.begin(); for (int j = 0; desc != tuple_descs.end(); ++desc, ++j) { Tuple* tuple = row->get_tuple(j); @@ -624,7 +624,7 @@ int RowBatch::total_byte_size() { continue; } result += (*desc)->byte_size(); - vector::const_iterator slot = (*desc)->string_slots().begin(); + std::vector::const_iterator slot = (*desc)->string_slots().begin(); for (; slot != (*desc)->string_slots().end(); ++slot) { DCHECK((*slot)->type().is_string_type()); if (tuple->is_null((*slot)->null_indicator_offset())) { diff --git a/be/src/tools/meta_tool.cpp b/be/src/tools/meta_tool.cpp index 916dd92875..05ffa09789 100644 --- a/be/src/tools/meta_tool.cpp +++ b/be/src/tools/meta_tool.cpp @@ -191,7 +191,7 @@ void batch_delete_meta(const std::string& tablet_file) { std::unordered_map> dir_map; while (std::getline(infile, line)) { total_num++; - vector v = strings::Split(line, ","); + std::vector v = strings::Split(line, ","); if (v.size() != 3) { std::cout << "invalid line in tablet_file: " << line << std::endl; err_num++; @@ -263,7 +263,7 @@ Status get_segment_footer(RandomAccessFile* input_file, SegmentFooterPB* footer) RETURN_IF_ERROR(input_file->size(&file_size)); if (file_size < 12) { - return Status::Corruption(Substitute("Bad segment file $0: file size $1 < 12", file_name, file_size)); + return Status::Corruption(strings::Substitute("Bad segment file $0: file size $1 < 12", file_name, file_size)); } uint8_t fixed_buf[12]; @@ -273,14 +273,14 @@ Status get_segment_footer(RandomAccessFile* input_file, SegmentFooterPB* footer) const char* k_segment_magic = "D0R1"; const uint32_t k_segment_magic_length = 4; if (memcmp(fixed_buf + 8, k_segment_magic, k_segment_magic_length) != 0) { - return Status::Corruption(Substitute("Bad segment file $0: magic number not match", file_name)); + return Status::Corruption(strings::Substitute("Bad segment file $0: magic number not match", file_name)); } // read footer PB uint32_t footer_length = doris::decode_fixed32_le(fixed_buf); if (file_size < 12 + footer_length) { return Status::Corruption( - Substitute("Bad segment file $0: file size $1 < $2", file_name, file_size, 12 + footer_length)); + strings::Substitute("Bad segment file $0: file size $1 < $2", file_name, file_size, 12 + footer_length)); } std::string footer_buf; footer_buf.resize(footer_length); @@ -291,13 +291,13 @@ Status get_segment_footer(RandomAccessFile* input_file, SegmentFooterPB* footer) uint32_t actual_checksum = doris::crc32c::Value(footer_buf.data(), footer_buf.size()); if (actual_checksum != expect_checksum) { return Status::Corruption( - Substitute("Bad segment file $0: footer checksum not match, actual=$1 vs expect=$2", + strings::Substitute("Bad segment file $0: footer checksum not match, actual=$1 vs expect=$2", file_name, actual_checksum, expect_checksum)); } // deserialize footer PB if (!footer->ParseFromString(footer_buf)) { - return Status::Corruption(Substitute("Bad segment file $0: failed to parse SegmentFooterPB", file_name)); + return Status::Corruption(strings::Substitute("Bad segment file $0: failed to parse SegmentFooterPB", file_name)); } return Status::OK(); } diff --git a/be/src/util/arrow/row_batch.cpp b/be/src/util/arrow/row_batch.cpp index c799e3bbe6..4caef4987b 100644 --- a/be/src/util/arrow/row_batch.cpp +++ b/be/src/util/arrow/row_batch.cpp @@ -83,7 +83,7 @@ Status convert_to_arrow_type(const TypeDescriptor& type, *result = std::make_shared(27, 9); break; default: - return Status::InvalidArgument(Substitute("Unknown primitive type($0)", type.type)); + return Status::InvalidArgument(strings::Substitute("Unknown primitive type($0)", type.type)); } return Status::OK(); } @@ -133,7 +133,7 @@ Status convert_to_doris_type(const arrow::DataType& type, builder->type(TYPE_DOUBLE); break; default: - return Status::InvalidArgument(Substitute("Unknown arrow type id($0)", type.id())); + return Status::InvalidArgument(strings::Substitute("Unknown arrow type id($0)", type.id())); } return Status::OK(); } diff --git a/be/src/util/arrow/row_block.cpp b/be/src/util/arrow/row_block.cpp index b6e60bc2f3..ca78b4a761 100644 --- a/be/src/util/arrow/row_block.cpp +++ b/be/src/util/arrow/row_block.cpp @@ -59,7 +59,7 @@ Status convert_to_arrow_type(FieldType type, std::shared_ptr *r *result = arrow::float64(); break; default: - return Status::InvalidArgument(Substitute("Unknown FieldType($0)", type)); + return Status::InvalidArgument(strings::Substitute("Unknown FieldType($0)", type)); } return Status::OK(); } @@ -69,7 +69,7 @@ Status convert_to_arrow_field(uint32_t cid, std::shared_ptr* result) { std::shared_ptr type; RETURN_IF_ERROR(convert_to_arrow_type(field->type(), &type)); - *result = arrow::field(Substitute("Col$0", cid), type, field->is_nullable()); + *result = arrow::field(strings::Substitute("Col$0", cid), type, field->is_nullable()); return Status::OK(); } @@ -108,7 +108,7 @@ Status convert_to_type_name(const arrow::DataType& type, *name = "DOUBLE"; break; default: - return Status::InvalidArgument(Substitute("Unknown arrow type id($0)", type.id())); + return Status::InvalidArgument(strings::Substitute("Unknown arrow type id($0)", type.id())); } return Status::OK(); } diff --git a/be/src/util/block_compression.cpp b/be/src/util/block_compression.cpp index cc631e1bf9..b65119d40a 100644 --- a/be/src/util/block_compression.cpp +++ b/be/src/util/block_compression.cpp @@ -54,7 +54,7 @@ public: LZ4_compress_default(input.data, output->data, input.size, output->size); if (compressed_len == 0) { return Status::InvalidArgument( - Substitute("Output buffer's capacity is not enough, size=$0", output->size)); + strings::Substitute("Output buffer's capacity is not enough, size=$0", output->size)); } output->size = compressed_len; return Status::OK(); @@ -65,7 +65,7 @@ public: LZ4_decompress_safe(input.data, output->data, input.size, output->size); if (decompressed_len < 0) { return Status::InvalidArgument( - Substitute("fail to do LZ4 decompress, error=$0", decompressed_len)); + strings::Substitute("fail to do LZ4 decompress, error=$0", decompressed_len)); } output->size = decompressed_len; return Status::OK(); @@ -91,7 +91,7 @@ public: LZ4F_compressFrame(output->data, output->size, input.data, input.size, &_s_preferences); if (LZ4F_isError(compressed_len)) { return Status::InvalidArgument( - Substitute("Fail to do LZ4F compress frame, msg=$0", + strings::Substitute("Fail to do LZ4F compress frame, msg=$0", LZ4F_getErrorName(compressed_len))); } output->size = compressed_len; @@ -103,7 +103,7 @@ public: auto lres = LZ4F_createCompressionContext(&ctx, LZ4F_VERSION); if (lres != 0) { return Status::InvalidArgument( - Substitute("Fail to do LZ4F compress, res=$0", LZ4F_getErrorName(lres))); + strings::Substitute("Fail to do LZ4F compress, res=$0", LZ4F_getErrorName(lres))); } auto st = _compress(ctx, inputs, output); LZ4F_freeCompressionContext(ctx); @@ -115,7 +115,7 @@ public: auto lres = LZ4F_createDecompressionContext(&ctx, LZ4F_VERSION); if (LZ4F_isError(lres)) { return Status::InvalidArgument( - Substitute("Fail to do LZ4F decompress, res=$0", LZ4F_getErrorName(lres))); + strings::Substitute("Fail to do LZ4F decompress, res=$0", LZ4F_getErrorName(lres))); } auto st = _decompress(ctx, input, output); LZ4F_freeDecompressionContext(ctx); @@ -132,7 +132,7 @@ private: auto wbytes = LZ4F_compressBegin(ctx, output->data, output->size, &_s_preferences); if (LZ4F_isError(wbytes)) { return Status::InvalidArgument( - Substitute("Fail to do LZ4F compress begin, res=$0", LZ4F_getErrorName(wbytes))); + strings::Substitute("Fail to do LZ4F compress begin, res=$0", LZ4F_getErrorName(wbytes))); } size_t offset = wbytes; for (auto input : inputs) { @@ -142,7 +142,7 @@ private: nullptr); if (LZ4F_isError(wbytes)) { return Status::InvalidArgument( - Substitute("Fail to do LZ4F compress update, res=$0", LZ4F_getErrorName(wbytes))); + strings::Substitute("Fail to do LZ4F compress update, res=$0", LZ4F_getErrorName(wbytes))); } offset += wbytes; } @@ -151,7 +151,7 @@ private: nullptr); if (LZ4F_isError(wbytes)) { return Status::InvalidArgument( - Substitute("Fail to do LZ4F compress end, res=$0", LZ4F_getErrorName(wbytes))); + strings::Substitute("Fail to do LZ4F compress end, res=$0", LZ4F_getErrorName(wbytes))); } offset += wbytes; output->size = offset; @@ -165,14 +165,14 @@ private: LZ4F_decompress(ctx, output->data, &output->size, input.data, &input_size, nullptr); if (LZ4F_isError(lres)) { return Status::InvalidArgument( - Substitute("Fail to do LZ4F decompress, res=$0", LZ4F_getErrorName(lres))); + strings::Substitute("Fail to do LZ4F decompress, res=$0", LZ4F_getErrorName(lres))); } else if (input_size != input.size) { return Status::InvalidArgument( - Substitute("Fail to do LZ4F decompress: trailing data left in compressed data, read=$0 vs given=$1", + strings::Substitute("Fail to do LZ4F decompress: trailing data left in compressed data, read=$0 vs given=$1", input_size, input.size)); } else if (lres != 0) { return Status::InvalidArgument( - Substitute("Fail to do LZ4F decompress: expect more compressed data, expect=$0", lres)); + strings::Substitute("Fail to do LZ4F decompress: expect more compressed data, expect=$0", lres)); } return Status::OK(); } @@ -306,7 +306,7 @@ public: auto zres = ::compress((Bytef*)output->data, &output->size, (Bytef*)input.data, input.size); if (zres != Z_OK) { return Status::InvalidArgument( - Substitute("Fail to do ZLib compress, error=$0", zError(zres))); + strings::Substitute("Fail to do ZLib compress, error=$0", zError(zres))); } return Status::OK(); } @@ -319,7 +319,7 @@ public: auto zres = deflateInit(&zstrm, Z_DEFAULT_COMPRESSION); if (zres != Z_OK) { return Status::InvalidArgument( - Substitute("Fail to do ZLib stream compress, error=$0, res=$1", + strings::Substitute("Fail to do ZLib stream compress, error=$0, res=$1", zError(zres), zres)); } // we assume that output is e @@ -336,7 +336,7 @@ public: zres = deflate(&zstrm, flush); if (zres != Z_OK && zres != Z_STREAM_END) { return Status::InvalidArgument( - Substitute("Fail to do ZLib stream compress, error=$0, res=$1", + strings::Substitute("Fail to do ZLib stream compress, error=$0, res=$1", zError(zres), zres)); } } @@ -345,7 +345,7 @@ public: zres = deflateEnd(&zstrm); if (zres != Z_OK) { return Status::InvalidArgument( - Substitute("Fail to do deflateEnd on ZLib stream, error=$0, res=$1", + strings::Substitute("Fail to do deflateEnd on ZLib stream, error=$0, res=$1", zError(zres), zres)); } return Status::OK(); @@ -356,7 +356,7 @@ public: auto zres = ::uncompress2((Bytef*)output->data, &output->size, (Bytef*)input.data, &input_size); if (zres != Z_OK) { return Status::InvalidArgument( - Substitute("Fail to do ZLib decompress, error=$0", zError(zres))); + strings::Substitute("Fail to do ZLib decompress, error=$0", zError(zres))); } return Status::OK(); } @@ -386,7 +386,7 @@ Status get_block_compression_codec( *codec = ZlibBlockCompression::instance(); break; default: - return Status::NotFound(Substitute("unknown compression type($0)", type)); + return Status::NotFound(strings::Substitute("unknown compression type($0)", type)); } return Status::OK(); } diff --git a/be/src/util/cgroup_util.cpp b/be/src/util/cgroup_util.cpp index 43b0ec1ae5..4fd6316d99 100644 --- a/be/src/util/cgroup_util.cpp +++ b/be/src/util/cgroup_util.cpp @@ -30,7 +30,7 @@ #include "util/file_utils.h" #include "util/string_parser.hpp" -#include "common/names.h" + using strings::CUnescape; using strings::Split; @@ -40,14 +40,14 @@ using std::pair; namespace doris { Status CGroupUtil::find_global_cgroup(const string& subsystem, string* path) { - ifstream proc_cgroups("/proc/self/cgroup", ios::in); + std::ifstream proc_cgroups("/proc/self/cgroup", std::ios::in); string line; while (true) { if (proc_cgroups.fail()) { - return Status::IOError(Substitute("Error reading /proc/self/cgroup: $0", get_str_err_msg())); + return Status::IOError(strings::Substitute("Error reading /proc/self/cgroup: $0", get_str_err_msg())); } else if (proc_cgroups.peek() == std::ifstream::traits_type::eof()) { return Status::NotFound( - Substitute("Could not find subsystem $0 in /proc/self/cgroup", subsystem)); + strings::Substitute("Could not find subsystem $0 in /proc/self/cgroup", subsystem)); } // The line format looks like this: // 4:memory:/user.slice @@ -57,16 +57,16 @@ Status CGroupUtil::find_global_cgroup(const string& subsystem, string* path) { if (!proc_cgroups.good()) { continue; } - vector fields = Split(line, ":"); + std::vector fields = Split(line, ":"); // ":" in the path does not appear to be escaped - bail in the unusual case that // we get too many tokens. if (fields.size() != 3) { return Status::InvalidArgument( - Substitute( + strings::Substitute( "Could not parse line from /proc/self/cgroup - had $0 > 3 tokens: '$1'", fields.size(), line)); } - vector subsystems = Split(fields[1], ","); + std::vector subsystems = Split(fields[1], ","); auto it = std::find(subsystems.begin(), subsystems.end(), subsystem); if (it != subsystems.end()) { *path = move(fields[2]); @@ -78,17 +78,17 @@ Status CGroupUtil::find_global_cgroup(const string& subsystem, string* path) { static Status unescape_path(const string& escaped, string* unescaped) { string err; if (!CUnescape(escaped, unescaped, &err)) { - return Status::InvalidArgument(Substitute("Could not unescape path '$0': $1", escaped, err)); + return Status::InvalidArgument(strings::Substitute("Could not unescape path '$0': $1", escaped, err)); } return Status::OK(); } static Status read_cgroup_value(const string& limit_file_path, int64_t* val) { - ifstream limit_file(limit_file_path, ios::in); + std::ifstream limit_file(limit_file_path, std::ios::in); string line; getline(limit_file, line); if (limit_file.fail() || limit_file.bad()) { - return Status::IOError(Substitute("Error reading $0: $1", limit_file_path, get_str_err_msg())); + return Status::IOError(strings::Substitute("Error reading $0: $1", limit_file_path, get_str_err_msg())); } StringParser::ParseResult pr; // Parse into an an int64_t If it overflows, returning the max value of int64_t is ok because that @@ -96,21 +96,21 @@ static Status read_cgroup_value(const string& limit_file_path, int64_t* val) { *val = StringParser::string_to_int(line.c_str(), line.size(), &pr); if ((pr != StringParser::PARSE_SUCCESS && pr != StringParser::PARSE_OVERFLOW)) { return Status::InvalidArgument( - Substitute("Failed to parse $0 as int64: '$1'", limit_file_path, line)); + strings::Substitute("Failed to parse $0 as int64: '$1'", limit_file_path, line)); } return Status::OK(); } Status CGroupUtil::find_cgroup_mounts( const string& subsystem, pair* result) { - ifstream mountinfo("/proc/self/mountinfo", ios::in); + std::ifstream mountinfo("/proc/self/mountinfo", std::ios::in); string line; while (true) { if (mountinfo.fail() || mountinfo.bad()) { - return Status::IOError(Substitute("Error reading /proc/self/mountinfo: $0", get_str_err_msg())); + return Status::IOError(strings::Substitute("Error reading /proc/self/mountinfo: $0", get_str_err_msg())); } else if (mountinfo.eof()) { return Status::NotFound( - Substitute("Could not find subsystem $0 in /proc/self/mountinfo", + strings::Substitute("Could not find subsystem $0 in /proc/self/mountinfo", subsystem)); } // The relevant lines look like below (see proc manpage for full documentation). The @@ -123,16 +123,16 @@ Status CGroupUtil::find_cgroup_mounts( // ro,nosuid,nodev,noexec,relatime master:15 - cgroup cgroup rw,memory getline(mountinfo, line); if (!mountinfo.good()) continue; - vector fields = Split(line, " ", SkipWhitespace()); + std::vector fields = Split(line, " ", SkipWhitespace()); if (fields.size() < 7) { return Status::InvalidArgument( - Substitute( + strings::Substitute( "Could not parse line from /proc/self/mountinfo - had $0 > 7 tokens: '$1'", fields.size(), line)); } if (fields[fields.size() - 3] != "cgroup") continue; // This is a cgroup mount. Check if it's the mount we're looking for. - vector cgroup_opts = Split(fields[fields.size() - 1], ",", SkipWhitespace()); + std::vector cgroup_opts = Split(fields[fields.size() - 1], ",", SkipWhitespace()); auto it = std::find(cgroup_opts.begin(), cgroup_opts.end(), subsystem); if (it == cgroup_opts.end()) { continue; @@ -157,7 +157,7 @@ Status CGroupUtil::find_abs_cgroup_path(const string& subsystem, string* path) { const string& system_path = paths.second; if (path->compare(0, system_path.size(), system_path) != 0) { return Status::InvalidArgument( - Substitute("Expected CGroup path '$0' to start with '$1'", + strings::Substitute("Expected CGroup path '$0' to start with '$1'", *path, system_path)); } path->replace(0, system_path.size(), mount_path); @@ -209,7 +209,7 @@ std::string CGroupUtil::debug_string() { int64_t mem_limit; Status status = find_cgroup_mem_limit(&mem_limit); if (status.ok()) { - mem_limit_str = Substitute("$0", mem_limit); + mem_limit_str = strings::Substitute("$0", mem_limit); } else { mem_limit_str = status.get_error_msg(); } @@ -227,7 +227,7 @@ std::string CGroupUtil::debug_string() { } else { cpu_limit_str = status.get_error_msg(); } - return Substitute("Process CGroup Info: memory.limit_in_bytes=$0, cpu cfs limits: $1", mem_limit_str, cpu_limit_str); + return strings::Substitute("Process CGroup Info: memory.limit_in_bytes=$0, cpu cfs limits: $1", mem_limit_str, cpu_limit_str); } bool CGroupUtil::enable() { diff --git a/be/src/util/cpu_info.cpp b/be/src/util/cpu_info.cpp index 650e41aae8..eceeb32279 100755 --- a/be/src/util/cpu_info.cpp +++ b/be/src/util/cpu_info.cpp @@ -53,7 +53,7 @@ #include "util/pretty_printer.h" #include "util/string_parser.hpp" -#include "common/names.h" + using boost::algorithm::contains; using boost::algorithm::trim; @@ -70,12 +70,12 @@ namespace doris { // first line. If the file cannot be opened, no error is reported. void WarnIfFileNotEqual( const string& filename, const string& expected, const string& warning_text) { - ifstream file(filename); + std::ifstream file(filename); if (!file) return; string line; getline(file, line); if (line != expected) { - LOG(ERROR) << "Expected " << expected << ", actual " << line << endl << warning_text; + LOG(ERROR) << "Expected " << expected << ", actual " << line << std::endl << warning_text; } } } // end anonymous namespace @@ -88,11 +88,11 @@ int64_t CpuInfo::original_hardware_flags_; int64_t CpuInfo::cycles_per_ms_; int CpuInfo::num_cores_ = 1; int CpuInfo::max_num_cores_ = 1; -string CpuInfo::model_name_ = "unknown"; +std::string CpuInfo::model_name_ = "unknown"; int CpuInfo::max_num_numa_nodes_; -unique_ptr CpuInfo::core_to_numa_node_; -vector> CpuInfo::numa_node_to_cores_; -vector CpuInfo::numa_node_core_idx_; +std::unique_ptr CpuInfo::core_to_numa_node_; +std::vector> CpuInfo::numa_node_to_cores_; +std::vector CpuInfo::numa_node_core_idx_; static struct { string name; @@ -132,7 +132,7 @@ void CpuInfo::init() { int num_cores = 0; // Read from /proc/cpuinfo - ifstream cpuinfo("/proc/cpuinfo"); + std::ifstream cpuinfo("/proc/cpuinfo"); while (cpuinfo) { getline(cpuinfo, line); size_t colon = line.find(':'); @@ -221,7 +221,7 @@ void CpuInfo::_init_numa() { for (int core = 0; core < max_num_cores_; ++core) { bool found_numa_node = false; for (int node = 0; node < max_num_numa_nodes_; ++node) { - if (fs::exists(Substitute("/sys/devices/system/cpu/cpu$0/node$1", core, node))) { + if (fs::exists(strings::Substitute("/sys/devices/system/cpu/cpu$0/node$1", core, node))) { core_to_numa_node_[core] = node; found_numa_node = true; break; @@ -237,7 +237,7 @@ void CpuInfo::_init_numa() { } void CpuInfo::_init_fake_numa_for_test( - int max_num_numa_nodes, const vector& core_to_numa_node) { + int max_num_numa_nodes, const std::vector& core_to_numa_node) { DCHECK_EQ(max_num_cores_, core_to_numa_node.size()); max_num_numa_nodes_ = max_num_numa_nodes; for (int i = 0; i < max_num_cores_; ++i) { @@ -252,7 +252,7 @@ void CpuInfo::_init_numa_node_to_cores() { numa_node_to_cores_.resize(max_num_numa_nodes_); numa_node_core_idx_.resize(max_num_cores_); for (int core = 0; core < max_num_cores_; ++core) { - vector* cores_of_node = &numa_node_to_cores_[core_to_numa_node_[core]]; + std::vector* cores_of_node = &numa_node_to_cores_[core_to_numa_node_[core]]; numa_node_core_idx_[core] = cores_of_node->size(); cores_of_node->push_back(core); } @@ -268,8 +268,8 @@ void CpuInfo::verify_cpu_requirements() { void CpuInfo::verify_performance_governor() { for (int cpu_id = 0; cpu_id < CpuInfo::num_cores(); ++cpu_id) { const string governor_file = - Substitute("/sys/devices/system/cpu/cpu$0/cpufreq/scaling_governor", cpu_id); - const string warning_text = Substitute( + strings::Substitute("/sys/devices/system/cpu/cpu$0/cpufreq/scaling_governor", cpu_id); + const string warning_text = strings::Substitute( "WARNING: CPU $0 is not using 'performance' governor. Note that changing the " "governor to 'performance' will reset the no_turbo setting to 0.", cpu_id); @@ -346,41 +346,41 @@ void CpuInfo::_get_cache_info(long cache_sizes[NUM_CACHE_LEVELS], #endif } -string CpuInfo::debug_string() { +std::string CpuInfo::debug_string() { DCHECK(initialized_); - stringstream stream; + std::stringstream stream; long cache_sizes[NUM_CACHE_LEVELS]; long cache_line_sizes[NUM_CACHE_LEVELS]; _get_cache_info(cache_sizes, cache_line_sizes); - string L1 = Substitute("L1 Cache: $0 (Line: $1)", + string L1 = strings::Substitute("L1 Cache: $0 (Line: $1)", PrettyPrinter::print(cache_sizes[L1_CACHE], TUnit::BYTES), PrettyPrinter::print(cache_line_sizes[L1_CACHE], TUnit::BYTES)); - string L2 = Substitute("L2 Cache: $0 (Line: $1)", + string L2 = strings::Substitute("L2 Cache: $0 (Line: $1)", PrettyPrinter::print(cache_sizes[L2_CACHE], TUnit::BYTES), PrettyPrinter::print(cache_line_sizes[L2_CACHE], TUnit::BYTES)); - string L3 = Substitute("L3 Cache: $0 (Line: $1)", + string L3 = strings::Substitute("L3 Cache: $0 (Line: $1)", PrettyPrinter::print(cache_sizes[L3_CACHE], TUnit::BYTES), PrettyPrinter::print(cache_line_sizes[L3_CACHE], TUnit::BYTES)); - stream << "Cpu Info:" << endl - << " Model: " << model_name_ << endl - << " Cores: " << num_cores_ << endl - << " Max Possible Cores: " << max_num_cores_ << endl - << " " << L1 << endl - << " " << L2 << endl - << " " << L3 << endl - << " Hardware Supports:" << endl; + stream << "Cpu Info:" << std::endl +<< " Model: " << model_name_ << std::endl +<< " Cores: " << num_cores_ << std::endl +<< " Max Possible Cores: " << max_num_cores_ << std::endl +<< " " << L1 << std::endl +<< " " << L2 << std::endl +<< " " << L3 << std::endl +<< " Hardware Supports:" << std::endl; for (int i = 0; i < num_flags; ++i) { if (is_supported(flag_mappings[i].flag)) { - stream << " " << flag_mappings[i].name << endl; + stream << " " << flag_mappings[i].name << std::endl; } } - stream << " Numa Nodes: " << max_num_numa_nodes_ << endl; + stream << " Numa Nodes: " << max_num_numa_nodes_ << std::endl; stream << " Numa Nodes of Cores:"; for (int core = 0; core < max_num_cores_; ++core) { stream << " " << core << "->" << core_to_numa_node_[core] << " |"; } - stream << endl; + stream << std::endl; return stream.str(); } diff --git a/be/src/util/file_utils.cpp b/be/src/util/file_utils.cpp index b8db85fa7a..0277fde57a 100644 --- a/be/src/util/file_utils.cpp +++ b/be/src/util/file_utils.cpp @@ -45,7 +45,7 @@ using strings::Substitute; Status FileUtils::create_dir(const std::string& path, Env* env) { if (path.empty()) { - return Status::InvalidArgument(Substitute("Unknown primitive type($0)", path)); + return Status::InvalidArgument(strings::Substitute("Unknown primitive type($0)", path)); } boost::filesystem::path p(path); diff --git a/be/src/util/os_info.cpp b/be/src/util/os_info.cpp index 33b4096d0e..a37afc8e58 100755 --- a/be/src/util/os_info.cpp +++ b/be/src/util/os_info.cpp @@ -25,12 +25,12 @@ #include -#include "common/names.h" + namespace doris { bool OsInfo::initialized_ = false; -string OsInfo::os_version_ = "Unknown"; +std::string OsInfo::os_version_ = "Unknown"; clockid_t OsInfo::fast_clock_ = CLOCK_MONOTONIC; std::string OsInfo::clock_name_ = "Unknown clocksource, clockid_t defaulting to CLOCK_MONOTONIC"; @@ -47,7 +47,7 @@ std::string OsInfo::clock_name_ = void OsInfo::Init() { DCHECK(!initialized_); // Read from /proc/version - ifstream version("/proc/version", ios::in); + std::ifstream version("/proc/version", std::ios::in); if (version.good()) getline(version, os_version_); if (version.is_open()) version.close(); @@ -73,11 +73,10 @@ void OsInfo::Init() { initialized_ = true; } -string OsInfo::DebugString() { +std::string OsInfo::DebugString() { DCHECK(initialized_); - stringstream stream; - stream << "OS version: " << os_version_ << endl - << "Clock: " << clock_name_ << endl; + std::stringstream stream; + stream << "OS version: " << os_version_ << std::endl << "Clock: " << clock_name_ << std::endl; return stream.str(); } diff --git a/be/src/util/os_util.cpp b/be/src/util/os_util.cpp index 15aed51d18..892f7cdce5 100644 --- a/be/src/util/os_util.cpp +++ b/be/src/util/os_util.cpp @@ -79,7 +79,7 @@ Status parse_stat(const std::string& buffer, std::string* name, ThreadStats* sta } string extracted_name = buffer.substr(open_paren + 1, close_paren - (open_paren + 1)); string rest = buffer.substr(close_paren + 2); - vector splits = Split(rest, " ", strings::SkipEmpty()); + std::vector splits = Split(rest, " ", strings::SkipEmpty()); if (splits.size() < kMaxOffset) { return Status::IOError("Unrecognised /proc format"); } @@ -107,7 +107,7 @@ Status get_thread_stats(int64_t tid, ThreadStats* stats) { } faststring buf; RETURN_IF_ERROR(env_util::read_file_to_string( - Env::Default(), Substitute("/proc/self/task/$0/stat", tid), &buf)); + Env::Default(), strings::Substitute("/proc/self/task/$0/stat", tid), &buf)); return parse_stat(buf.ToString(), nullptr, stats); } @@ -146,7 +146,7 @@ bool is_being_debugged() { return false; } StringPiece buf_sp(reinterpret_cast(buf.data()), buf.size()); - vector lines = Split(buf_sp, "\n"); + std::vector lines = Split(buf_sp, "\n"); for (const auto& l : lines) { if (!HasPrefixString(l, "TracerPid:")) continue; std::pair key_val = Split(l, "\t"); diff --git a/be/src/util/path_util.cpp b/be/src/util/path_util.cpp index 525261809b..a04f664172 100644 --- a/be/src/util/path_util.cpp +++ b/be/src/util/path_util.cpp @@ -37,7 +37,7 @@ namespace path_util { const string kTmpInfix = ".doristmp"; -string join_path_segments(const string& a, const string& b) { +std::string join_path_segments(const string& a, const string& b) { if (a.empty()) { return b; } else if (b.empty()) { @@ -47,23 +47,23 @@ string join_path_segments(const string& a, const string& b) { } } -vector join_path_segments_v(const vector& v, const string& s) { - vector out; +std::vector join_path_segments_v(const std::vector& v, const string& s) { + std::vector out; for (const string& path : v) { out.emplace_back(join_path_segments(path, s)); } return out; } -vector split_path(const string& path) { +std::vector split_path(const string& path) { if (path.empty()) { return {}; } - vector segments; + std::vector segments; if (path[0] == '/') { segments.emplace_back("/"); } - vector pieces = Split(path, "/", SkipEmpty()); + std::vector pieces = Split(path, "/", SkipEmpty()); for (const StringPiece& piece : pieces) { segments.emplace_back(piece.data(), piece.size()); } @@ -71,19 +71,19 @@ vector split_path(const string& path) { } // strdup use malloc to obtain memory for the new string, it should be freed with free. -// but unique_ptr use delete to free memory by default, so it should specify free memory using free +// but std::unique_ptr use delete to free memory by default, so it should specify free memory using free -string dir_name(const string& path) { +std::string dir_name(const string& path) { std::vector path_copy(path.c_str(), path.c_str() + path.size() + 1); return dirname(&path_copy[0]); } -string base_name(const string& path) { +std::string base_name(const string& path) { std::vector path_copy(path.c_str(), path.c_str() + path.size() + 1); return basename(&path_copy[0]); } -string file_extension(const string& path) { +std::string file_extension(const string& path) { string file_name = base_name(path); if (file_name == "." || file_name == "..") { return ""; diff --git a/be/src/util/perf_counters.cpp b/be/src/util/perf_counters.cpp index 7728970d1b..57f3390b6f 100644 --- a/be/src/util/perf_counters.cpp +++ b/be/src/util/perf_counters.cpp @@ -307,7 +307,7 @@ bool PerfCounters::get_sys_counters(vector& buffer) { // write_bytes: 0 // cancelled_write_bytes: 0 bool PerfCounters::get_proc_self_io_counters(vector& buffer) { - std::ifstream file("/proc/self/io", ios::in); + std::ifstream file("/proc/self/io", std::ios::in); std::string buf; int64_t values[PROC_IO_LAST_COUNTER]; int ret = 0; @@ -347,7 +347,7 @@ bool PerfCounters::get_proc_self_io_counters(vector& buffer) { } bool PerfCounters::get_proc_self_status_counters(vector& buffer) { - ifstream file("/proc/self/status", ios::in); + std::ifstream file("/proc/self/status", std::ios::in); string buf; while (file) { @@ -467,12 +467,12 @@ void PerfCounters::snapshot(const string& name) { string fixed_name = name; if (fixed_name.size() == 0) { - stringstream ss; + std::stringstream ss; ss << _snapshots.size() + 1; fixed_name = ss.str(); } - vector buffer(_counters.size()); + std::vector buffer(_counters.size()); get_sys_counters(buffer); get_proc_self_io_counters(buffer); @@ -482,7 +482,7 @@ void PerfCounters::snapshot(const string& name) { _snapshot_names.push_back(fixed_name); } -const vector* PerfCounters::counters(int snapshot) const { +const std::vector* PerfCounters::counters(int snapshot) const { if (snapshot < 0 || snapshot >= _snapshots.size()) { return NULL; } @@ -498,7 +498,7 @@ void PerfCounters::pretty_print(ostream* s) const { stream << setw(PRETTY_PRINT_WIDTH) << _counter_names[i]; } - stream << endl; + stream << std::endl; for (int s = 0; s < _snapshots.size(); s++) { stream << setw(8) << _snapshot_names[s]; @@ -509,10 +509,10 @@ void PerfCounters::pretty_print(ostream* s) const { << PrettyPrinter::print(snapshot[i], _counters[i].type); } - stream << endl; + stream << std::endl; } - stream << endl; + stream << std::endl; } } diff --git a/be/src/util/thread.cpp b/be/src/util/thread.cpp index b7cb63192c..d2400be3d1 100644 --- a/be/src/util/thread.cpp +++ b/be/src/util/thread.cpp @@ -184,7 +184,7 @@ void ThreadMgr::display_thread_callback(const WebPageHandler::ArgumentMap& args, // The critical section is as short as possible so as to minimize the delay // imposed on new threads that acquire the lock in write mode. - vector descriptors_to_print; + std::vector descriptors_to_print; if (!requested_all) { MutexLock l(&_lock); const auto* category = FindOrNull(_thread_categories, *category_name); @@ -210,7 +210,7 @@ void ThreadMgr::display_thread_callback(const WebPageHandler::ArgumentMap& args, } } else { // List all thread groups and the number of threads running in each. - vector> thread_categories_info; + std::vector> thread_categories_info; uint64_t running; { MutexLock l(&_lock); diff --git a/be/src/util/threadpool.cpp b/be/src/util/threadpool.cpp index 582f224c4e..ed5400f887 100644 --- a/be/src/util/threadpool.cpp +++ b/be/src/util/threadpool.cpp @@ -254,7 +254,7 @@ ThreadPool::ThreadPool(const ThreadPoolBuilder& builder) ThreadPool::~ThreadPool() { // There should only be one live token: the one used in tokenless submission. - CHECK_EQ(1, _tokens.size()) << Substitute( + CHECK_EQ(1, _tokens.size()) << strings::Substitute( "Threadpool $0 destroyed with $1 allocated tokens", _name, _tokens.size()); shutdown(); @@ -343,7 +343,7 @@ std::unique_ptr ThreadPool::new_token(ExecutionMode mode) { void ThreadPool::release_token(ThreadPoolToken* t) { MutexLock unique_lock(&_lock); - CHECK(!t->is_active()) << Substitute("Token with state $0 may not be released", + CHECK(!t->is_active()) << strings::Substitute("Token with state $0 may not be released", ThreadPoolToken::state_to_string(t->state())); CHECK_EQ(1, _tokens.erase(t)); } @@ -376,7 +376,7 @@ Status ThreadPool::do_submit(std::shared_ptr r, ThreadPoolToken* token - _total_queued_tasks; if (capacity_remaining < 1) { return Status::ServiceUnavailable( - Substitute("Thread pool is at capacity ($0/$1 tasks running, $2/$3 tasks queued)", + strings::Substitute("Thread pool is at capacity ($0/$1 tasks running, $2/$3 tasks queued)", _num_threads + _num_threads_pending_start, _max_threads, _total_queued_tasks, _max_queue_size)); } @@ -599,14 +599,14 @@ void ThreadPool::dispatch_thread() { } Status ThreadPool::create_thread() { - return Thread::create("thread pool", Substitute("$0 [worker]", _name), + return Thread::create("thread pool", strings::Substitute("$0 [worker]", _name), &ThreadPool::dispatch_thread, this, nullptr); } void ThreadPool::check_not_pool_thread_unlocked() { Thread* current = Thread::current_thread(); if (ContainsKey(_threads, current)) { - LOG(FATAL) << Substitute("Thread belonging to thread pool '$0' with " + LOG(FATAL) << strings::Substitute("Thread belonging to thread pool '$0' with " "name '$1' called pool function that would result in deadlock", _name, current->name()); } diff --git a/be/src/util/time.cpp b/be/src/util/time.cpp index dce9071671..9f0061f9a4 100755 --- a/be/src/util/time.cpp +++ b/be/src/util/time.cpp @@ -57,7 +57,7 @@ static string TimepointToString(const chrono::system_clock::time_point& t, // i.e., times before the Unix epoch. static string FormatSubSecond(const chrono::system_clock::time_point& t, TimePrecision p) { - stringstream ss; + std::stringstream ss; auto frac = t.time_since_epoch(); if (p == TimePrecision::Millisecond) { auto subsec = chrono::duration_cast(frac) % MILLIS_PER_SEC; @@ -82,7 +82,7 @@ static string FormatSubSecond(const chrono::system_clock::time_point& t, static string ToString(const chrono::system_clock::time_point& t, TimePrecision p, bool utc) { - stringstream ss; + std::stringstream ss; ss << TimepointToString(t, utc); ss << FormatSubSecond(t, p); return ss.str(); @@ -106,32 +106,32 @@ static chrono::system_clock::time_point TimepointFromUnixMicros(int64_t us) { return chrono::system_clock::time_point(chrono::microseconds(us)); } -string doris::ToStringFromUnix(int64_t s, TimePrecision p) { +std::string doris::ToStringFromUnix(int64_t s, TimePrecision p) { chrono::system_clock::time_point t = TimepointFromUnix(s); return ToString(t, p, false); } -string doris::ToUtcStringFromUnix(int64_t s, TimePrecision p) { +std::string doris::ToUtcStringFromUnix(int64_t s, TimePrecision p) { chrono::system_clock::time_point t = TimepointFromUnix(s); return ToString(t, p, true); } -string doris::ToStringFromUnixMillis(int64_t ms, TimePrecision p) { +std::string doris::ToStringFromUnixMillis(int64_t ms, TimePrecision p) { chrono::system_clock::time_point t = TimepointFromUnixMillis(ms); return ToString(t, p, false); } -string doris::ToUtcStringFromUnixMillis(int64_t ms, TimePrecision p) { +std::string doris::ToUtcStringFromUnixMillis(int64_t ms, TimePrecision p) { chrono::system_clock::time_point t = TimepointFromUnixMillis(ms); return ToString(t, p, true); } -string doris::ToStringFromUnixMicros(int64_t us, TimePrecision p) { +std::string doris::ToStringFromUnixMicros(int64_t us, TimePrecision p) { chrono::system_clock::time_point t = TimepointFromUnixMicros(us); return ToString(t, p, false); } -string doris::ToUtcStringFromUnixMicros(int64_t us, TimePrecision p) { +std::string doris::ToUtcStringFromUnixMicros(int64_t us, TimePrecision p) { chrono::system_clock::time_point t = TimepointFromUnixMicros(us); return ToString(t, p, true); } diff --git a/be/src/util/trace.cpp b/be/src/util/trace.cpp index 6033a1bbe1..6b941b0e89 100644 --- a/be/src/util/trace.cpp +++ b/be/src/util/trace.cpp @@ -140,8 +140,8 @@ void Trace::Dump(std::ostream* out, int flags) const { // enough that we aren't worried about stalling concurrent tracers // (whereas doing the logging itself while holding the lock might be // too slow, if the output stream is a file, for example). - vector entries; - vector>> child_traces; + std::vector entries; + std::vector>> child_traces; { std::lock_guard l(lock_); for (TraceEntry* cur = entries_head_; @@ -193,13 +193,13 @@ void Trace::Dump(std::ostream* out, int flags) const { out->flags(save_flags); } -string Trace::DumpToString(int flags) const { +std::string Trace::DumpToString(int flags) const { std::ostringstream s; Dump(&s, flags); return s.str(); } -string Trace::MetricsAsJSON() const { +std::string Trace::MetricsAsJSON() const { // TODO(yingchun): simplify implement here, we could import JsonWriter in the future. rapidjson::StringBuffer buf; rapidjson::Writer jw(buf); @@ -220,7 +220,7 @@ void Trace::MetricsToJSON(rapidjson::Writer* jw) const jw->String(e.first.c_str()); jw->Int64(e.second); } - vector>> child_traces; + std::vector>> child_traces; { std::lock_guard l(lock_); child_traces = child_traces_; diff --git a/be/src/util/zlib.cpp b/be/src/util/zlib.cpp index fe0c325b4e..980e841843 100644 --- a/be/src/util/zlib.cpp +++ b/be/src/util/zlib.cpp @@ -30,7 +30,7 @@ using std::ostream; using std::string; -using std::unique_ptr; + #define ZRETURN_NOT_OK(call) \ RETURN_IF_ERROR(ZlibResultToStatus(call)) @@ -81,7 +81,7 @@ Status CompressLevel(Slice input, int level, ostream* out) { zs.avail_in = input.get_size(); zs.next_in = (unsigned char*)(input.mutable_data()); const int kChunkSize = 256 * 1024; - unique_ptr chunk(new unsigned char[kChunkSize]); + std::unique_ptr chunk(new unsigned char[kChunkSize]); int flush; do { zs.avail_out = kChunkSize; diff --git a/be/test/agent/agent_server_test.cpp b/be/test/agent/agent_server_test.cpp index b10463d06a..fd2d519d85 100644 --- a/be/test/agent/agent_server_test.cpp +++ b/be/test/agent/agent_server_test.cpp @@ -34,7 +34,7 @@ namespace doris { TEST(SubmitTasksTest, TestSubmitTasks){ TAgentResult return_value; - vector tasks; + std::vector tasks; ExecEnv env; TMasterInfo master_info; diff --git a/be/test/exec/hash_table_test.cpp b/be/test/exec/hash_table_test.cpp index c01aca3e5a..4e5f0d4115 100644 --- a/be/test/exec/hash_table_test.cpp +++ b/be/test/exec/hash_table_test.cpp @@ -44,8 +44,8 @@ public: protected: ObjectPool _pool; MemPool _mem_pool; - vector _build_expr; - vector _probe_expr; + std::vector _build_expr; + std::vector _probe_expr; virtual void SetUp() { RowDescriptor desc; @@ -106,7 +106,7 @@ protected: struct ProbeTestData { TupleRow* probe_row; - vector expected_build_rows; + std::vector expected_build_rows; }; void probe_test(HashTable* table, ProbeTestData* data, int num_data, bool scan) { @@ -234,7 +234,7 @@ TEST_F(HashTableTest, BasicTest) { TEST_F(HashTableTest, ScanTest) { HashTable hash_table(_build_expr, _probe_expr, 1, false, 0); // Add 1 row with val 1, 2 with val 2, etc - vector build_rows; + std::vector build_rows; ProbeTestData probe_rows[15]; probe_rows[0].probe_row = create_tuple_row(0); diff --git a/be/test/exec/new_olap_scan_node_test.cpp b/be/test/exec/new_olap_scan_node_test.cpp index a9e3982d1c..8e1b9d4c06 100644 --- a/be/test/exec/new_olap_scan_node_test.cpp +++ b/be/test/exec/new_olap_scan_node_test.cpp @@ -237,13 +237,13 @@ public: } } - void read_data(int version, vector* data) { + void read_data(int version, std::vector* data) { data->clear(); int row[21]; for (int i = 0; i <= version; ++i) { - stringstream ss; + std::stringstream ss; ss << "./testrun/case3/_fc_dayhour" << i << ".txt"; fstream f(ss.str()); @@ -256,7 +256,7 @@ public: break; } - stringstream str; + std::stringstream str; str << "[("; str << row[0] << " "; str << row[2] << " "; @@ -277,7 +277,7 @@ private: ObjectPool _obj_pool; DescriptorTbl* _desc_tbl; RuntimeState _runtime_stat; - vector _scan_ranges; + std::vector _scan_ranges; }; TEST_F(TestOlapScanNode, SimpleTest) { @@ -307,7 +307,7 @@ TEST_F(TestOlapScanNode, SimpleTest) { TEST_F(TestOlapScanNode, MultiColumnSingleVersionTest) { _scan_ranges[0].scan_range.doris_scan_range.__set_version("0"); _scan_ranges[0].scan_range.doris_scan_range.__set_version_hash("0"); - vector data; + std::vector data; read_data(0, &data); OlapScanNode scan_node(&_obj_pool, _tnode, *_desc_tbl); @@ -345,7 +345,7 @@ TEST_F(TestOlapScanNode, MultiColumnSingleVersionTest) { TEST_F(TestOlapScanNode, MultiColumnMultiVersionTest) { _scan_ranges[0].scan_range.doris_scan_range.__set_version("9"); _scan_ranges[0].scan_range.doris_scan_range.__set_version_hash("0"); - vector data; + std::vector data; read_data(9, &data); OlapScanNode scan_node(&_obj_pool, _tnode, *_desc_tbl); diff --git a/be/test/olap/delete_handler_test.cpp b/be/test/olap/delete_handler_test.cpp index 12db222ea1..0bf0d6e5de 100644 --- a/be/test/olap/delete_handler_test.cpp +++ b/be/test/olap/delete_handler_test.cpp @@ -850,7 +850,7 @@ TEST_F(TestDeleteHandler, InitSuccess) { res = _delete_handler.init(tablet->tablet_schema(), tablet->delete_predicates(), 4); ASSERT_EQ(OLAP_SUCCESS, res); ASSERT_EQ(4, _delete_handler.conditions_num()); - vector conds_version = _delete_handler.get_conds_version(); + std::vector conds_version = _delete_handler.get_conds_version(); EXPECT_EQ(4, conds_version.size()); sort(conds_version.begin(), conds_version.end()); EXPECT_EQ(1, conds_version[0]); @@ -894,7 +894,7 @@ TEST_F(TestDeleteHandler, FilterDataSubconditions) { ASSERT_EQ(1, _delete_handler.conditions_num()); // 构造一行测试数据 - vector data_str; + std::vector data_str; data_str.push_back("1"); data_str.push_back("6"); data_str.push_back("8"); @@ -980,7 +980,7 @@ TEST_F(TestDeleteHandler, FilterDataConditions) { ASSERT_EQ(OLAP_SUCCESS, res); ASSERT_EQ(3, _delete_handler.conditions_num()); - vector data_str; + std::vector data_str; data_str.push_back("4"); data_str.push_back("5"); data_str.push_back("8"); @@ -1046,7 +1046,7 @@ TEST_F(TestDeleteHandler, FilterDataVersion) { ASSERT_EQ(2, _delete_handler.conditions_num()); // 构造一行测试数据 - vector data_str; + std::vector data_str; data_str.push_back("1"); data_str.push_back("6"); data_str.push_back("8"); diff --git a/be/test/olap/fs/file_block_manager_test.cpp b/be/test/olap/fs/file_block_manager_test.cpp index 6c99253cd6..4be0c7d168 100644 --- a/be/test/olap/fs/file_block_manager_test.cpp +++ b/be/test/olap/fs/file_block_manager_test.cpp @@ -31,7 +31,7 @@ namespace doris { class FileBlockManagerTest : public testing::Test { protected: - const string kBlockManagerDir = "./ut_dir/file_block_manager"; + const std::string kBlockManagerDir = "./ut_dir/file_block_manager"; void SetUp() override { if (FileUtils::check_exist(kBlockManagerDir)) { @@ -55,12 +55,12 @@ TEST_F(FileBlockManagerTest, NormalTest) { std::unique_ptr fbm(new fs::FileBlockManager(env, std::move(bm_opts))); std::unique_ptr wblock; - string fname = kBlockManagerDir + "/test_file"; + std::string fname = kBlockManagerDir + "/test_file"; fs::CreateBlockOptions wblock_opts({ fname }); Status st = fbm->create_block(wblock_opts, &wblock); ASSERT_TRUE(st.ok()) << st.get_error_msg(); - string data = "abcdefghijklmnopqrstuvwxyz"; + std::string data = "abcdefghijklmnopqrstuvwxyz"; wblock->append(data); wblock->close(); @@ -69,7 +69,7 @@ TEST_F(FileBlockManagerTest, NormalTest) { uint64_t file_size = 0; ASSERT_TRUE(rblock->size(&file_size).ok()); ASSERT_EQ(data.size(), file_size); - string read_buff(data.size(), 'a'); + std::string read_buff(data.size(), 'a'); Slice read_slice(read_buff); rblock->read(0, read_slice); ASSERT_EQ(data, read_buff); diff --git a/be/test/olap/memory/column_delta_test.cpp b/be/test/olap/memory/column_delta_test.cpp index 9a31ae876b..2193e09d2b 100644 --- a/be/test/olap/memory/column_delta_test.cpp +++ b/be/test/olap/memory/column_delta_test.cpp @@ -40,7 +40,7 @@ TEST(ColumnDelta, Index) { size_t nblock = num_block(BaseSize, Column::BLOCK_SIZE); ASSERT_TRUE(delta->alloc(nblock, updates.size(), sizeof(uint32_t), false).ok()); DeltaIndex* index = delta->index(); - vector& block_ends = index->block_ends(); + std::vector& block_ends = index->block_ends(); Buffer& idxdata = index->_data; Buffer& data = delta->data(); uint32_t cidx = 0; diff --git a/be/test/olap/memory/column_test.cpp b/be/test/olap/memory/column_test.cpp index 83accd0598..90411ed22d 100644 --- a/be/test/olap/memory/column_test.cpp +++ b/be/test/olap/memory/column_test.cpp @@ -38,7 +38,7 @@ struct ColumnTest { scoped_refptr c(new Column(cs, CT, 1)); std::unique_ptr writer; ASSERT_TRUE(c->create_writer(&writer).ok()); - vector values(InsertCount, 0); + std::vector values(InsertCount, 0); for (size_t i = 0; i < values.size(); i++) { values[i] = (CppType)rand(); EXPECT_TRUE(writer->insert((uint32_t)i, &values[i]).ok()); @@ -60,7 +60,7 @@ struct ColumnTest { scoped_refptr c(new Column(cs, CT, 1)); std::unique_ptr writer; ASSERT_TRUE(c->create_writer(&writer).ok()); - vector values(InsertCount, 0); + std::vector values(InsertCount, 0); for (size_t i = 0; i < values.size(); i++) { values[i] = (CppType)rand(); if (is_null(values[i])) { @@ -94,7 +94,7 @@ struct ColumnTest { scoped_refptr c(new Column(cs, CT, 1)); std::unique_ptr writer; ASSERT_TRUE(c->create_writer(&writer).ok()); - vector values(InsertCount, 0); + std::vector values(InsertCount, 0); for (size_t i = 0; i < values.size(); i++) { values[i] = (CppType)rand(); EXPECT_TRUE(writer->insert((uint32_t)i, &values[i]).ok()); @@ -105,7 +105,7 @@ struct ColumnTest { scoped_refptr oldc = c; for (size_t u = 0; u < UpdateTime; u++) { ASSERT_TRUE(c->create_writer(&writer).ok()); - vector update_idxs; + std::vector update_idxs; for (size_t i = 0; i < UpdateCount; i++) { uint32_t idx = rand() % values.size(); //CppType oldv = values[idx]; @@ -138,7 +138,7 @@ struct ColumnTest { scoped_refptr c(new Column(cs, CT, 1)); std::unique_ptr writer; ASSERT_TRUE(c->create_writer(&writer).ok()); - vector values(InsertCount, 0); + std::vector values(InsertCount, 0); for (size_t i = 0; i < values.size(); i++) { values[i] = (CppType)rand(); if (is_null(values[i])) { @@ -154,7 +154,7 @@ struct ColumnTest { scoped_refptr oldc = c; for (size_t u = 0; u < UpdateTime; u++) { ASSERT_TRUE(c->create_writer(&writer).ok()); - vector update_idxs; + std::vector update_idxs; for (size_t i = 0; i < UpdateCount; i++) { uint32_t idx = rand() % values.size(); //CppType oldv = values[idx]; diff --git a/be/test/olap/memory/mem_tablet_test.cpp b/be/test/olap/memory/mem_tablet_test.cpp index 979f4622da..9f0bff9619 100644 --- a/be/test/olap/memory/mem_tablet_test.cpp +++ b/be/test/olap/memory/mem_tablet_test.cpp @@ -73,7 +73,7 @@ TEST(MemTablet, writescan) { ASSERT_TRUE(tablet->init().ok()); uint64_t cur_version = 0; - vector alldata(num_insert); + std::vector alldata(num_insert); // insert srand(1); @@ -101,7 +101,7 @@ TEST(MemTablet, writescan) { EXPECT_TRUE(writer.set("city", city % 2 == 0 ? nullptr : &city).ok()); EXPECT_TRUE(writer.end_row().ok()); } - vector wtxn_buff; + std::vector wtxn_buff; EXPECT_TRUE(writer.finish_batch(&wtxn_buff).ok()); PartialRowBatch* batch = wtx->new_batch(); EXPECT_TRUE(batch->load(std::move(wtxn_buff)).ok()); @@ -131,7 +131,7 @@ TEST(MemTablet, writescan) { EXPECT_TRUE(writer.set("city", city % 2 == 0 ? nullptr : &city).ok()); EXPECT_TRUE(writer.end_row().ok()); } - vector wtxn_buff; + std::vector wtxn_buff; EXPECT_TRUE(writer.finish_batch(&wtxn_buff).ok()); PartialRowBatch* batch = wtx->new_batch(); EXPECT_TRUE(batch->load(std::move(wtxn_buff)).ok()); diff --git a/be/test/olap/memory/partial_row_batch_test.cpp b/be/test/olap/memory/partial_row_batch_test.cpp index f031cc9369..33861dd84b 100644 --- a/be/test/olap/memory/partial_row_batch_test.cpp +++ b/be/test/olap/memory/partial_row_batch_test.cpp @@ -50,7 +50,7 @@ TEST(PartialRowbatch, write) { } EXPECT_TRUE(writer.end_row().ok()); } - vector buffer; + std::vector buffer; writer.finish_batch(&buffer); PartialRowBatch rb(&sc); diff --git a/be/test/olap/olap_snapshot_converter_test.cpp b/be/test/olap/olap_snapshot_converter_test.cpp index 112756c494..1237b2a28c 100644 --- a/be/test/olap/olap_snapshot_converter_test.cpp +++ b/be/test/olap/olap_snapshot_converter_test.cpp @@ -129,7 +129,7 @@ TEST_F(OlapSnapshotConverterTest, ToNewAndToOldSnapshot) { ASSERT_TRUE(ret); OlapSnapshotConverter converter; TabletMetaPB tablet_meta_pb; - vector pending_rowsets; + std::vector pending_rowsets; OLAPStatus status = converter.to_new_snapshot(header_msg, _tablet_data_path, _tablet_data_path, &tablet_meta_pb, &pending_rowsets, true); ASSERT_TRUE(status == OLAP_SUCCESS); diff --git a/be/test/olap/rowset/beta_rowset_test.cpp b/be/test/olap/rowset/beta_rowset_test.cpp index 43f0ac6559..05862e88bd 100644 --- a/be/test/olap/rowset/beta_rowset_test.cpp +++ b/be/test/olap/rowset/beta_rowset_test.cpp @@ -70,7 +70,7 @@ protected: ExecEnv* exec_env = doris::ExecEnv::GetInstance(); exec_env->set_storage_engine(k_engine); - const string rowset_dir = "./data_test/data/beta_rowset_test"; + const std::string rowset_dir = "./data_test/data/beta_rowset_test"; ASSERT_TRUE(FileUtils::create_dir(rowset_dir).ok()); } diff --git a/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp b/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp index c0481af3d3..39ebaa7b5b 100644 --- a/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp +++ b/be/test/olap/rowset/segment_v2/column_reader_writer_test.cpp @@ -38,7 +38,7 @@ using std::string; namespace doris { namespace segment_v2 { -static const string TEST_DIR = "./ut_dir/column_reader_writer_test"; +static const std::string TEST_DIR = "./ut_dir/column_reader_writer_test"; class ColumnReaderWriterTest : public testing::Test { public: @@ -65,14 +65,14 @@ private: }; template -void test_nullable_data(uint8_t* src_data, uint8_t* src_is_null, int num_rows, string test_name) { +void test_nullable_data(uint8_t* src_data, uint8_t* src_is_null, int num_rows, std::string test_name) { using Type = typename TypeTraits::CppType; Type* src = (Type*)src_data; ColumnMetaPB meta; // write data - string fname = TEST_DIR + "/" + test_name; + std::string fname = TEST_DIR + "/" + test_name; { std::unique_ptr wblock; fs::CreateBlockOptions opts({ fname }); @@ -217,7 +217,7 @@ void test_nullable_data(uint8_t* src_data, uint8_t* src_is_null, int num_rows, s } template -void test_array_nullable_data(Collection* src_data, uint8_t* src_is_null, int num_rows, string test_name) { +void test_array_nullable_data(Collection* src_data, uint8_t* src_is_null, int num_rows, std::string test_name) { Collection* src = src_data; ColumnMetaPB meta; TabletColumn list_column(OLAP_FIELD_AGGREGATION_NONE, OLAP_FIELD_TYPE_ARRAY); @@ -230,7 +230,7 @@ void test_array_nullable_data(Collection* src_data, uint8_t* src_is_null, int nu Field* field = FieldFactory::create(list_column); // write data - string fname = TEST_DIR + "/" + test_name; + std::string fname = TEST_DIR + "/" + test_name; { std::unique_ptr wblock; fs::CreateBlockOptions opts({ fname }); @@ -603,48 +603,48 @@ TEST_F(ColumnReaderWriterTest, test_types) { } TEST_F(ColumnReaderWriterTest, test_default_value) { - string v_int("1"); + std::string v_int("1"); int32_t result = 1; test_read_default_value(v_int, &result); test_read_default_value(v_int, &result); test_read_default_value(v_int, &result); - string v_bigint("9223372036854775807"); + std::string v_bigint("9223372036854775807"); int64_t result_bigint = std::numeric_limits::max(); test_read_default_value(v_bigint, &result_bigint); int128_t result_largeint = std::numeric_limits::max(); test_read_default_value(v_bigint, &result_largeint); - string v_float("1.00"); + std::string v_float("1.00"); float result2 = 1.00; test_read_default_value(v_float, &result2); - string v_double("1.00"); + std::string v_double("1.00"); double result3 = 1.00; test_read_default_value(v_double, &result3); - string v_varchar("varchar"); + std::string v_varchar("varchar"); test_read_default_value(v_varchar, &v_varchar); - string v_char("char"); + std::string v_char("char"); test_read_default_value(v_char, &v_char); char* c = (char *)malloc(1); c[0] = 0; - string v_object(c, 1); + std::string v_object(c, 1); test_read_default_value(v_object, &v_object); test_read_default_value(v_object, &v_object); free(c); - string v_date("2019-11-12"); + std::string v_date("2019-11-12"); uint24_t result_date(1034092); test_read_default_value(v_date, &result_date); - string v_datetime("2019-11-12 12:01:08"); + std::string v_datetime("2019-11-12 12:01:08"); int64_t result_datetime = 20191112120108; test_read_default_value(v_datetime, &result_datetime); - string v_decimal("102418.000000002"); + std::string v_decimal("102418.000000002"); decimal12_t decimal(102418, 2); test_read_default_value(v_decimal, &decimal); } diff --git a/be/test/olap/rowset/segment_v2/segment_test.cpp b/be/test/olap/rowset/segment_v2/segment_test.cpp index 1d12365c3a..5f327acfed 100644 --- a/be/test/olap/rowset/segment_v2/segment_test.cpp +++ b/be/test/olap/rowset/segment_v2/segment_test.cpp @@ -47,7 +47,7 @@ namespace segment_v2 { using std::string; using std::shared_ptr; -using std::unique_ptr; + using std::vector; using ValueGenerator = std::function; @@ -84,7 +84,7 @@ protected: } } - TabletSchema create_schema(const vector& columns, int num_short_key_columns = -1) { + TabletSchema create_schema(const std::vector& columns, int num_short_key_columns = -1) { TabletSchema res; int num_key_columns = 0; for (auto& col : columns) { @@ -108,7 +108,7 @@ protected: static int seg_id = 0; // must use unique filename for each segment, otherwise page cache kicks in and produces // the wrong answer (it use (filename,offset) as cache key) - string filename = strings::Substitute("$0/seg_$1.dat", kSegmentDir, seg_id++); + std::string filename = strings::Substitute("$0/seg_$1.dat", kSegmentDir, seg_id++); std::unique_ptr wblock; fs::CreateBlockOptions block_opts({ filename }); Status st = fs::fs_util::block_manager()->create_block(block_opts, &wblock); @@ -140,7 +140,7 @@ protected: ASSERT_EQ(nrows, (*res)->num_rows()); } private: - const string kSegmentDir = "./ut_dir/segment_test"; + const std::string kSegmentDir = "./ut_dir/segment_test"; }; TEST_F(SegmentReaderWriterTest, normal) { @@ -300,15 +300,15 @@ TEST_F(SegmentReaderWriterTest, LazyMaterialization) { // lazy enabled when predicate is subset of returned columns: // select c1, c2 where c2 = 30; Schema read_schema(tablet_schema); - unique_ptr predicate(new EqualPredicate(1, 30)); - const vector predicates = {predicate.get()}; + std::unique_ptr predicate(new EqualPredicate(1, 30)); + const std::vector predicates = {predicate.get()}; OlapReaderStatistics stats; StorageReadOptions read_opts; read_opts.column_predicates = &predicates; read_opts.stats = &stats; - unique_ptr iter; + std::unique_ptr iter; ASSERT_TRUE(segment->new_iterator(read_schema, read_opts, &iter).ok()); RowBlockV2 block(read_schema, 1024); @@ -323,16 +323,16 @@ TEST_F(SegmentReaderWriterTest, LazyMaterialization) { // lazy disabled when all return columns have predicates: // select c1, c2 where c1 = 10 and c2 = 100; Schema read_schema(tablet_schema); - unique_ptr p0(new EqualPredicate(0, 10)); - unique_ptr p1(new EqualPredicate(1, 100)); - const vector predicates = {p0.get(), p1.get()}; + std::unique_ptr p0(new EqualPredicate(0, 10)); + std::unique_ptr p1(new EqualPredicate(1, 100)); + const std::vector predicates = {p0.get(), p1.get()}; OlapReaderStatistics stats; StorageReadOptions read_opts; read_opts.column_predicates = &predicates; read_opts.stats = &stats; - unique_ptr iter; + std::unique_ptr iter; ASSERT_TRUE(segment->new_iterator(read_schema, read_opts, &iter).ok()); RowBlockV2 block(read_schema, 1024); @@ -346,13 +346,13 @@ TEST_F(SegmentReaderWriterTest, LazyMaterialization) { { // lazy disabled when no predicate: // select c2 - vector read_cols = {1}; + std::vector read_cols = {1}; Schema read_schema(tablet_schema.columns(), read_cols); OlapReaderStatistics stats; StorageReadOptions read_opts; read_opts.stats = &stats; - unique_ptr iter; + std::unique_ptr iter; ASSERT_TRUE(segment->new_iterator(read_schema, read_opts, &iter).ok()); RowBlockV2 block(read_schema, 1024); @@ -376,15 +376,15 @@ TEST_F(SegmentReaderWriterTest, LazyMaterialization) { // lazy disabled when all predicates are removed by bitmap index: // select c1, c2 where c2 = 30; Schema read_schema(tablet_schema); - unique_ptr predicate(new EqualPredicate(0, 20)); - const vector predicates = { predicate.get() }; + std::unique_ptr predicate(new EqualPredicate(0, 20)); + const std::vector predicates = { predicate.get() }; OlapReaderStatistics stats; StorageReadOptions read_opts; read_opts.column_predicates = &predicates; read_opts.stats = &stats; - unique_ptr iter; + std::unique_ptr iter; ASSERT_TRUE(segment->new_iterator(read_schema, read_opts, &iter).ok()); RowBlockV2 block(read_schema, 1024); @@ -649,12 +649,12 @@ TEST_F(SegmentReaderWriterTest, estimate_segment_size) { } TEST_F(SegmentReaderWriterTest, TestDefaultValueColumn) { - vector columns = { create_int_key(1), create_int_key(2), create_int_value(3), create_int_value(4) }; + std::vector columns = { create_int_key(1), create_int_key(2), create_int_value(3), create_int_value(4) }; TabletSchema build_schema = create_schema(columns); // add a column with null default value { - vector read_columns = columns; + std::vector read_columns = columns; read_columns.push_back(create_int_value(5, OLAP_FIELD_AGGREGATION_SUM, true, "NULL")); TabletSchema query_schema = create_schema(read_columns); @@ -703,7 +703,7 @@ TEST_F(SegmentReaderWriterTest, TestDefaultValueColumn) { // add a column with non-null default value { - vector read_columns = columns; + std::vector read_columns = columns; read_columns.push_back(create_int_value(5, OLAP_FIELD_AGGREGATION_SUM, true, "10086")); TabletSchema query_schema = create_schema(read_columns); diff --git a/be/test/olap/tablet_mgr_test.cpp b/be/test/olap/tablet_mgr_test.cpp index aaf18c5e06..ea99d9ac69 100644 --- a/be/test/olap/tablet_mgr_test.cpp +++ b/be/test/olap/tablet_mgr_test.cpp @@ -119,7 +119,7 @@ TEST_F(TabletMgrTest, CreateTablet) { create_tablet_req.__set_tablet_id(111); create_tablet_req.__set_version(2); create_tablet_req.__set_version_hash(3333); - vector data_dirs; + std::vector data_dirs; data_dirs.push_back(_data_dir); OLAPStatus create_st = _tablet_mgr->create_tablet(create_tablet_req, data_dirs); ASSERT_TRUE(create_st == OLAP_SUCCESS); @@ -180,7 +180,7 @@ TEST_F(TabletMgrTest, CreateTabletWithSequence) { create_tablet_req.__set_tablet_id(111); create_tablet_req.__set_version(2); create_tablet_req.__set_version_hash(3333); - vector data_dirs; + std::vector data_dirs; data_dirs.push_back(_data_dir); OLAPStatus create_st = _tablet_mgr->create_tablet(create_tablet_req, data_dirs); ASSERT_TRUE(create_st == OLAP_SUCCESS); @@ -215,7 +215,7 @@ TEST_F(TabletMgrTest, DropTablet) { create_tablet_req.__set_tablet_id(111); create_tablet_req.__set_version(2); create_tablet_req.__set_version_hash(3333); - vector data_dirs; + std::vector data_dirs; data_dirs.push_back(_data_dir); OLAPStatus create_st = _tablet_mgr->create_tablet(create_tablet_req, data_dirs); ASSERT_TRUE(create_st == OLAP_SUCCESS); diff --git a/be/test/runtime/buffered_block_mgr2_test.cpp b/be/test/runtime/buffered_block_mgr2_test.cpp index 94ca7a0dee..81810e9eae 100644 --- a/be/test/runtime/buffered_block_mgr2_test.cpp +++ b/be/test/runtime/buffered_block_mgr2_test.cpp @@ -85,10 +85,10 @@ protected: } // Reinitialize _test_env to have multiple temporary directories. - vector InitMultipleTmpDirs(int num_dirs) { - vector tmp_dirs; + std::vector InitMultipleTmpDirs(int num_dirs) { + std::vector tmp_dirs; for (int i = 0; i < num_dirs; ++i) { - stringstream dir_str; + std::stringstream dir_str; dir_str << "/tmp/buffered-block-mgr-test." << i; const string& dir = dir_str.str(); // Fix permissions in case old directories were left from previous runs of test. @@ -159,8 +159,8 @@ protected: void CreateMgrsAndClients(int64_t start_query_id, int num_mgrs, int buffers_per_mgr, int block_size, int reserved_blocks_per_client, const std::shared_ptr& tracker, - vector* mgrs, - vector* clients) { + std::vector* mgrs, + std::vector* clients) { for (int i = 0; i < num_mgrs; ++i) { BufferedBlockMgr2::Client* client; BufferedBlockMgr2* mgr = @@ -179,7 +179,7 @@ protected: } void AllocateBlocks(BufferedBlockMgr2* block_mgr, BufferedBlockMgr2::Client* client, - int num_blocks, vector* blocks) { + int num_blocks, std::vector* blocks) { int32_t* data = NULL; Status status; BufferedBlockMgr2::Block* new_block; @@ -194,7 +194,7 @@ protected: } // Pin all blocks, expecting they are pinned successfully. - void PinBlocks(const vector& blocks) { + void PinBlocks(const std::vector& blocks) { for (int i = 0; i < blocks.size(); ++i) { bool pinned = false; EXPECT_TRUE(blocks[i]->pin(&pinned).ok()); @@ -203,20 +203,20 @@ protected: } // Pin all blocks, expecting no errors from unpin() calls. - void UnpinBlocks(const vector& blocks) { + void UnpinBlocks(const std::vector& blocks) { for (int i = 0; i < blocks.size(); ++i) { EXPECT_TRUE(blocks[i]->unpin().ok()); } } static void WaitForWrites(BufferedBlockMgr2* block_mgr) { - vector block_mgrs; + std::vector block_mgrs; block_mgrs.push_back(block_mgr); WaitForWrites(block_mgrs); } // Wait for writes issued through block managers to complete. - static void WaitForWrites(const vector& block_mgrs) { + static void WaitForWrites(const std::vector& block_mgrs) { int max_attempts = WRITE_WAIT_MILLIS / WRITE_CHECK_INTERVAL_MILLIS; for (int i = 0; i < max_attempts; ++i) { SleepFor(MonoDelta::FromMilliseconds(WRITE_CHECK_INTERVAL_MILLIS)); @@ -227,7 +227,7 @@ protected: EXPECT_TRUE(false) << "Writes did not complete after " << WRITE_WAIT_MILLIS << "ms"; } - static bool AllWritesComplete(const vector& block_mgrs) { + static bool AllWritesComplete(const std::vector& block_mgrs) { for (int i = 0; i < block_mgrs.size(); ++i) { RuntimeProfile::Counter* writes_outstanding = block_mgrs[i]->profile()->get_counter("BlockWritesOutstanding"); @@ -255,7 +255,7 @@ protected: // Find a block in the list that is backed by a file with the given directory as prefix // of its path. static BufferedBlockMgr2::Block* FindBlockForDir( - const vector& blocks, const string& dir) { + const std::vector& blocks, const string& dir) { for (int i = 0; i < blocks.size(); ++i) { if (BlockInDir(blocks[i], dir)) { return blocks[i]; @@ -321,7 +321,7 @@ protected: RuntimeProfile* profile = block_mgr->profile(); RuntimeProfile::Counter* buffered_pin = profile->get_counter("BufferedPins"); - vector blocks; + std::vector blocks; AllocateBlocks(block_mgr, client, max_num_buffers, &blocks); EXPECT_EQ(block_mgr->bytes_allocated(), max_num_buffers * block_size); @@ -385,9 +385,9 @@ protected: const int iters_before_close = num_iterations - 5000; bool close_called = false; unordered_map pinned_block_map; - vector> pinned_blocks; + std::vector> pinned_blocks; unordered_map unpinned_block_map; - vector> unpinned_blocks; + std::vector> unpinned_blocks; typedef enum { Pin, New, Unpin, Delete, Close } ApiFunction; ApiFunction api_function; @@ -563,9 +563,9 @@ protected: workers.join_all(); } - scoped_ptr _test_env; + boost::scoped_ptr _test_env; std::shared_ptr _client_tracker; - vector _created_tmp_dirs; + std::vector _created_tmp_dirs; }; TEST_F(BufferedBlockMgrTest, get_new_block) { @@ -584,7 +584,7 @@ TEST_F(BufferedBlockMgrTest, GetNewBlockSmallBlocks) { CreateMgrAndClient(0, max_num_blocks, block_size, 0, _client_tracker, &client); EXPECT_EQ(0, _test_env->block_mgr_parent_tracker()->consumption()); - vector blocks; + std::vector blocks; // Allocate a small block. BufferedBlockMgr2::Block* new_block = NULL; @@ -644,7 +644,7 @@ TEST_F(BufferedBlockMgrTest, Pin) { block_mgr = CreateMgrAndClient(0, max_num_blocks, block_size, 0, _client_tracker, &client); - vector blocks; + std::vector blocks; AllocateBlocks(block_mgr, client, max_num_blocks, &blocks); // Unpin them all. @@ -704,7 +704,7 @@ TEST_F(BufferedBlockMgrTest, Deletion) { RuntimeProfile::Counter* recycled_cnt = profile->get_counter("BlocksRecycled"); RuntimeProfile::Counter* created_cnt = profile->get_counter("BlocksCreated"); - vector blocks; + std::vector blocks; AllocateBlocks(block_mgr, client, max_num_buffers, &blocks); EXPECT_TRUE(created_cnt->value() == max_num_buffers); @@ -776,7 +776,7 @@ TEST_F(BufferedBlockMgrTest, Close) { block_mgr = CreateMgrAndClient(0, max_num_buffers, block_size, 0, _client_tracker, &client); - vector blocks; + std::vector blocks; AllocateBlocks(block_mgr, client, max_num_buffers, &blocks); block_mgr->cancel(); @@ -818,7 +818,7 @@ TEST_F(BufferedBlockMgrTest, WriteError) { block_mgr = CreateMgrAndClient(0, max_num_buffers, block_size, 0, _client_tracker, &client); - vector blocks; + std::vector blocks; AllocateBlocks(block_mgr, client, max_num_buffers, &blocks); // Unpin two blocks here, to ensure that backing storage is allocated in tmp file. for (int i = 0; i < 2; ++i) { @@ -862,7 +862,7 @@ TEST_F(BufferedBlockMgrTest, TmpFileAllocateError) { BufferedBlockMgr2* block_mgr = CreateMgrAndClient(0, max_num_buffers, _block_size, 0, _client_tracker, &client); - vector blocks; + std::vector blocks; AllocateBlocks(block_mgr, client, max_num_buffers, &blocks); // Unpin a block, forcing a write. status = blocks[0]->unpin(); @@ -887,21 +887,21 @@ TEST_F(BufferedBlockMgrTest, TmpFileAllocateError) { TEST_F(BufferedBlockMgrTest, DISABLED_WriteErrorBlacklist) { // TEST_F(BufferedBlockMgrTest, WriteErrorBlacklist) { // Set up two buffered block managers with two temporary dirs. - vector tmp_dirs = InitMultipleTmpDirs(2); + std::vector tmp_dirs = InitMultipleTmpDirs(2); // Simulate two concurrent queries. const int NUM_BLOCK_MGRS = 2; const int MAX_NUM_BLOCKS = 4; int blocks_per_mgr = MAX_NUM_BLOCKS / NUM_BLOCK_MGRS; - vector block_mgrs; - vector clients; + std::vector block_mgrs; + std::vector clients; CreateMgrsAndClients(0, NUM_BLOCK_MGRS, blocks_per_mgr, _block_size, 0, _client_tracker, &block_mgrs, &clients); // Allocate files for all 2x2 combinations by unpinning blocks. - vector> blocks; - vector all_blocks; + std::vector> blocks; + std::vector all_blocks; for (int i = 0; i < NUM_BLOCK_MGRS; ++i) { - vector mgr_blocks; + std::vector mgr_blocks; AllocateBlocks(block_mgrs[i], clients[i], blocks_per_mgr, &mgr_blocks); UnpinBlocks(mgr_blocks); for (int j = 0; j < blocks_per_mgr; ++j) { @@ -926,7 +926,7 @@ TEST_F(BufferedBlockMgrTest, DISABLED_WriteErrorBlacklist) { EXPECT_TRUE(block_mgrs[error_mgr]->is_cancelled()); EXPECT_FALSE(block_mgrs[no_error_mgr]->is_cancelled()); // Temporary device with error should no longer be active. - vector active_tmp_devices = + std::vector active_tmp_devices = _test_env->tmp_file_mgr()->active_tmp_devices(); EXPECT_EQ(tmp_dirs.size() - 1, active_tmp_devices.size()); for (int i = 0; i < active_tmp_devices.size(); ++i) { @@ -942,7 +942,7 @@ TEST_F(BufferedBlockMgrTest, DISABLED_WriteErrorBlacklist) { EXPECT_TRUE(FindBlockForDir(blocks[no_error_mgr], good_dir) != NULL); EXPECT_TRUE(FindBlockForDir(blocks[no_error_mgr], error_dir) != NULL); // The second block manager should avoid using bad directory for new blocks. - vector no_error_new_blocks; + std::vector no_error_new_blocks; AllocateBlocks(block_mgrs[no_error_mgr], clients[no_error_mgr], blocks_per_mgr, &no_error_new_blocks); UnpinBlocks(no_error_new_blocks); @@ -955,7 +955,7 @@ TEST_F(BufferedBlockMgrTest, DISABLED_WriteErrorBlacklist) { BufferedBlockMgr2::Client* new_client; BufferedBlockMgr2* new_block_mgr = CreateMgrAndClient(9999, blocks_per_mgr, _block_size, 0, _client_tracker, &new_client); - vector new_mgr_blocks; + std::vector new_mgr_blocks; AllocateBlocks(new_block_mgr, new_client, blocks_per_mgr, &new_mgr_blocks); UnpinBlocks(new_mgr_blocks); for (int i = 0; i < blocks_per_mgr; ++i) { @@ -969,21 +969,21 @@ TEST_F(BufferedBlockMgrTest, DISABLED_WriteErrorBlacklist) { /// being allocated in other directories. TEST_F(BufferedBlockMgrTest, AllocationErrorHandling) { // Set up two buffered block managers with two temporary dirs. - vector tmp_dirs = InitMultipleTmpDirs(2); + std::vector tmp_dirs = InitMultipleTmpDirs(2); // Simulate two concurrent queries. int num_block_mgrs = 2; int max_num_blocks = 4; int blocks_per_mgr = max_num_blocks / num_block_mgrs; - // vector runtime_states; - vector block_mgrs; - vector clients; + // std::vector runtime_states; + std::vector block_mgrs; + std::vector clients; CreateMgrsAndClients(0, num_block_mgrs, blocks_per_mgr, _block_size, 0, _client_tracker, &block_mgrs, &clients); // Allocate files for all 2x2 combinations by unpinning blocks. - vector> blocks; + std::vector> blocks; for (int i = 0; i < num_block_mgrs; ++i) { - vector mgr_blocks; + std::vector mgr_blocks; LOG(INFO) << "Iter " << i; AllocateBlocks(block_mgrs[i], clients[i], blocks_per_mgr, &mgr_blocks); blocks.push_back(mgr_blocks); @@ -1012,12 +1012,12 @@ TEST_F(BufferedBlockMgrTest, AllocationErrorHandling) { // Test that block manager fails cleanly when all directories are inaccessible at runtime. TEST_F(BufferedBlockMgrTest, NoDirsAllocationError) { - vector tmp_dirs = InitMultipleTmpDirs(2); + std::vector tmp_dirs = InitMultipleTmpDirs(2); int max_num_buffers = 2; BufferedBlockMgr2::Client* client; BufferedBlockMgr2* block_mgr = CreateMgrAndClient(0, max_num_buffers, _block_size, 0, _client_tracker, &client); - vector blocks; + std::vector blocks; AllocateBlocks(block_mgr, client, max_num_buffers, &blocks); for (int i = 0; i < tmp_dirs.size(); ++i) { const string& tmp_scratch_subdir = tmp_dirs[i] + SCRATCH_SUFFIX; @@ -1055,7 +1055,7 @@ TEST_F(BufferedBlockMgrTest, MultipleClients) { reserved = block_mgr->try_acquire_tmp_reservation(client2, 1); EXPECT_TRUE(reserved); - vector client1_blocks; + std::vector client1_blocks; // Allocate all of client1's reserved blocks, they should all succeed. AllocateBlocks(block_mgr, client1, client1_buffers, &client1_blocks); @@ -1070,7 +1070,7 @@ TEST_F(BufferedBlockMgrTest, MultipleClients) { EXPECT_FALSE(reserved); // Allocate all of client2's reserved blocks, these should succeed. - vector client2_blocks; + std::vector client2_blocks; AllocateBlocks(block_mgr, client2, client2_buffers, &client2_blocks); // Try allocating one more from client 2, that should fail. @@ -1162,12 +1162,12 @@ TEST_F(BufferedBlockMgrTest, MultipleClientsExtraBuffers) { EXPECT_TRUE(status.ok()); EXPECT_TRUE(client2 != NULL); - vector client1_blocks; + std::vector client1_blocks; // Allocate all of client1's reserved blocks, they should all succeed. AllocateBlocks(block_mgr, client1, client1_buffers, &client1_blocks); // Allocate all of client2's reserved blocks, these should succeed. - vector client2_blocks; + std::vector client2_blocks; AllocateBlocks(block_mgr, client2, client2_buffers, &client2_blocks); // We have two spare buffers now. Each client should be able to allocate it. diff --git a/be/test/runtime/buffered_tuple_stream2_test.cpp b/be/test/runtime/buffered_tuple_stream2_test.cpp index 251e537624..46317c4be0 100644 --- a/be/test/runtime/buffered_tuple_stream2_test.cpp +++ b/be/test/runtime/buffered_tuple_stream2_test.cpp @@ -75,8 +75,8 @@ protected: } virtual void create_descriptors() { - vector nullable_tuples(1, false); - vector tuple_ids(1, static_cast(0)); + std::vector nullable_tuples(1, false); + std::vector tuple_ids(1, static_cast(0)); DescriptorTblBuilder int_builder(&_pool); int_builder.declare_tuple() << TYPE_INT; @@ -170,7 +170,7 @@ protected: return batch; } - void AppendRowTuples(TupleRow* row, vector* results) { + void AppendRowTuples(TupleRow* row, std::vector* results) { DCHECK(row != NULL); const int int_tuples = _int_desc->tuple_descriptors().size(); for (int i = 0; i < int_tuples; ++i) { @@ -178,7 +178,7 @@ protected: } } - void AppendRowTuples(TupleRow* row, vector* results) { + void AppendRowTuples(TupleRow* row, std::vector* results) { DCHECK(row != NULL); const int string_tuples = _string_desc->tuple_descriptors().size(); for (int i = 0; i < string_tuples; ++i) { @@ -186,7 +186,7 @@ protected: } } - void AppendValue(Tuple* t, vector* results) { + void AppendValue(Tuple* t, std::vector* results) { if (t == NULL) { // For the tests indicate null-ability using the max int value results->push_back(std::numeric_limits::max()); @@ -195,7 +195,7 @@ protected: } } - void AppendValue(Tuple* t, vector* results) { + void AppendValue(Tuple* t, std::vector* results) { if (t == NULL) { results->push_back(StringValue()); } else { @@ -209,7 +209,7 @@ protected: } template - void ReadValues(BufferedTupleStream2* stream, RowDescriptor* desc, vector* results, + void ReadValues(BufferedTupleStream2* stream, RowDescriptor* desc, std::vector* results, int num_batches = -1) { bool eos = false; RowBatch batch(*desc, BATCH_SIZE, _tracker.get()); @@ -225,7 +225,7 @@ protected: } while (!eos && (num_batches < 0 || batches_read <= num_batches)); } - virtual void VerifyResults(const vector& results, int exp_rows, bool gen_null) { + virtual void VerifyResults(const std::vector& results, int exp_rows, bool gen_null) { const int int_tuples = _int_desc->tuple_descriptors().size(); EXPECT_EQ(results.size(), exp_rows * int_tuples); for (int i = 0; i < exp_rows; ++i) { @@ -244,7 +244,7 @@ protected: } } - virtual void VerifyResults(const vector& results, int exp_rows, + virtual void VerifyResults(const std::vector& results, int exp_rows, bool gen_null) { const int string_tuples = _string_desc->tuple_descriptors().size(); EXPECT_EQ(results.size(), exp_rows * string_tuples); @@ -308,7 +308,7 @@ protected: ASSERT_TRUE(status.ok()); // Read all the rows back - vector results; + std::vector results; ReadValues(&stream, desc, &results); // Verify result @@ -329,7 +329,7 @@ protected: status = stream.unpin_stream(); ASSERT_TRUE(status.ok()); - vector results; + std::vector results; for (int i = 0; i < num_batches; ++i) { RowBatch* batch = CreateIntBatch(i * BATCH_SIZE, BATCH_SIZE, false); @@ -353,7 +353,7 @@ protected: } } - scoped_ptr _test_env; + boost::scoped_ptr _test_env; RuntimeState* _runtime_state; BufferedBlockMgr2::Client* _client; @@ -361,7 +361,7 @@ protected: ObjectPool _pool; RowDescriptor* _int_desc; RowDescriptor* _string_desc; - scoped_ptr _mem_pool; + boost::scoped_ptr _mem_pool; }; @@ -369,8 +369,8 @@ protected: class SimpleNullStreamTest : public SimpleTupleStreamTest { protected: virtual void create_descriptors() { - vector nullable_tuples(1, true); - vector tuple_ids(1, static_cast(0)); + std::vector nullable_tuples(1, true); + std::vector tuple_ids(1, static_cast(0)); DescriptorTblBuilder int_builder(&_pool); int_builder.declare_tuple() << TYPE_INT; @@ -388,12 +388,12 @@ protected: class MultiTupleStreamTest : public SimpleTupleStreamTest { protected: virtual void create_descriptors() { - vector nullable_tuples; + std::vector nullable_tuples; nullable_tuples.push_back(false); nullable_tuples.push_back(false); nullable_tuples.push_back(false); - vector tuple_ids; + std::vector tuple_ids; tuple_ids.push_back(static_cast(0)); tuple_ids.push_back(static_cast(1)); tuple_ids.push_back(static_cast(2)); @@ -418,12 +418,12 @@ protected: class MultiNullableTupleStreamTest : public SimpleTupleStreamTest { protected: virtual void create_descriptors() { - vector nullable_tuples; + std::vector nullable_tuples; nullable_tuples.push_back(false); nullable_tuples.push_back(true); nullable_tuples.push_back(true); - vector tuple_ids; + std::vector tuple_ids; tuple_ids.push_back(static_cast(0)); tuple_ids.push_back(static_cast(1)); tuple_ids.push_back(static_cast(2)); @@ -452,8 +452,8 @@ protected: virtual void create_descriptors() { // tuples: (array, array>) (array) - vector nullable_tuples(2, true); - vector tuple_ids; + std::vector nullable_tuples(2, true); + std::vector tuple_ids; tuple_ids.push_back(static_cast(0)); tuple_ids.push_back(static_cast(1)); TypeDescriptor string_array_type; @@ -555,7 +555,7 @@ TEST_F(SimpleTupleStreamTest, UnpinPin) { ASSERT_TRUE(status.ok()); ASSERT_TRUE(pinned); - vector results; + std::vector results; // Read and verify result a few times. We should be able to reread the stream if // we don't use delete on read mode. @@ -727,7 +727,7 @@ TEST_F(ArrayTupleStreamTest, TestArrayDeepCopy) { const int NUM_ROWS = 4000; BufferedTupleStream2 stream(_runtime_state, *_array_desc, _runtime_state->block_mgr2(), _client, false, false); - const vector& tuple_descs = _array_desc->tuple_descriptors(); + const std::vector& tuple_descs = _array_desc->tuple_descriptors(); // Write out a predictable pattern of data by iterating over arrays of constants. int strings_index = 0; // we take the mod of this as index into STRINGS. int array_lens[] = { 0, 1, 5, 10, 1000, 2, 49, 20 }; @@ -741,11 +741,11 @@ TEST_F(ArrayTupleStreamTest, TestArrayDeepCopy) { // malloc(tuple_descs[0]->byte_size()))); // gscoped_ptr tuple1(reinterpret_cast( // malloc(tuple_descs[1]->byte_size()))); - scoped_ptr row(reinterpret_cast( + boost::scoped_ptr row(reinterpret_cast( malloc(tuple_descs.size() * sizeof(Tuple*)))); - scoped_ptr tuple0(reinterpret_cast( + boost::scoped_ptr tuple0(reinterpret_cast( malloc(tuple_descs[0]->byte_size()))); - scoped_ptr tuple1(reinterpret_cast( + boost::scoped_ptr tuple1(reinterpret_cast( malloc(tuple_descs[1]->byte_size()))); memset(tuple0.get(), 0, tuple_descs[0]->byte_size()); memset(tuple1.get(), 0, tuple_descs[1]->byte_size()); diff --git a/be/test/runtime/data_stream_test.cpp b/be/test/runtime/data_stream_test.cpp index 527d2fde35..1134e0b16f 100644 --- a/be/test/runtime/data_stream_test.cpp +++ b/be/test/runtime/data_stream_test.cpp @@ -220,7 +220,7 @@ protected: string _stmt; // RowBatch generation - scoped_ptr _batch; + boost::scoped_ptr _batch; int _next_val; int64_t* _tuple_mem; @@ -232,7 +232,7 @@ protected: TDataStreamSink _broadcast_sink; TDataStreamSink _random_sink; TDataStreamSink _hash_sink; - vector _dest; + std::vector _dest; struct SenderInfo { thread* thread_handle; @@ -241,7 +241,7 @@ protected: SenderInfo() : thread_handle(NULL), num_bytes_sent(0) {} }; - vector _sender_info; + std::vector _sender_info; struct ReceiverInfo { TPartitionType::type stream_type; @@ -267,7 +267,7 @@ protected: stream_recvr.reset(); } }; - vector _receiver_info; + std::vector _receiver_info; // Create an instance id and add it to _dest void get_next_instance_id(TUniqueId* instance_id) { @@ -308,10 +308,10 @@ protected: EXPECT_TRUE(DescriptorTbl::create(&_obj_pool, thrift_desc_tbl, &_desc_tbl).ok()); _runtime_state.set_desc_tbl(_desc_tbl); - vector row_tids; + std::vector row_tids; row_tids.push_back(0); - vector nullable_tuples; + std::vector nullable_tuples; nullable_tuples.push_back(false); _row_desc = _obj_pool.add(new RowDescriptor(*_desc_tbl, row_tids, nullable_tuples)); } @@ -339,9 +339,9 @@ protected: _rhs_slot_ctx->open(NULL); SortExecExprs* sort_exprs = _obj_pool.add(new SortExecExprs()); sort_exprs->init(vector(1, _lhs_slot_ctx), - vector(1, _rhs_slot_ctx)); + std::vector(1, _rhs_slot_ctx)); _less_than = _obj_pool.add( - new TupleRowComparator(*sort_exprs, vector(1, true), vector(1, false))); + new TupleRowComparator(*sort_exprs, std::vector(1, true), std::vector(1, false))); } // Create _batch, but don't fill it with data yet. Assumes we created _row_desc. @@ -562,7 +562,7 @@ protected: EXPECT_TRUE(sender.prepare(&state).ok()); EXPECT_TRUE(sender.open(&state).ok()); - scoped_ptr batch(create_row_batch()); + boost::scoped_ptr batch(create_row_batch()); SenderInfo& info = _sender_info[sender_num]; int next_val = 0; diff --git a/be/test/runtime/disk_io_mgr_test.cpp b/be/test/runtime/disk_io_mgr_test.cpp index b6f99eff20..0590ba02fe 100644 --- a/be/test/runtime/disk_io_mgr_test.cpp +++ b/be/test/runtime/disk_io_mgr_test.cpp @@ -175,7 +175,7 @@ protected: return range; } - scoped_ptr _pool; + boost::scoped_ptr _pool; mutex _written_mutex; condition_variable _writes_done; @@ -200,7 +200,7 @@ TEST_F(DiskIoMgrTest, SingleWriter) { EXPECT_TRUE(false); } - scoped_ptr read_io_mgr(new DiskIoMgr(1, 1, 1, 10)); + boost::scoped_ptr read_io_mgr(new DiskIoMgr(1, 1, 1, 10)); std::shared_ptr reader_mem_tracker(new MemTracker(LARGE_MEM_LIMIT)); Status status = read_io_mgr->init(reader_mem_tracker); ASSERT_TRUE(status.ok()); @@ -318,7 +318,7 @@ TEST_F(DiskIoMgrTest, SingleWriterCancel) { EXPECT_TRUE(false); } - scoped_ptr read_io_mgr(new DiskIoMgr(1, 1, 1, 10)); + boost::scoped_ptr read_io_mgr(new DiskIoMgr(1, 1, 1, 10)); std::shared_ptr reader_mem_tracker(new MemTracker(LARGE_MEM_LIMIT)); Status status = read_io_mgr->init(reader_mem_tracker); ASSERT_TRUE(status.ok()); @@ -403,7 +403,7 @@ TEST_F(DiskIoMgrTest, SingleReader) { status = io_mgr.register_context(&reader, reader_mem_tracker); ASSERT_TRUE(status.ok()); - vector ranges; + std::vector ranges; for (int i = 0; i < len; ++i) { int disk_id = i % num_disks; ranges.push_back(init_range(num_buffers, tmp_file, 0, len, disk_id, @@ -460,8 +460,8 @@ TEST_F(DiskIoMgrTest, AddScanRangeTest) { status = io_mgr.register_context(&reader, reader_mem_tracker); ASSERT_TRUE(status.ok()); - vector ranges_first_half; - vector ranges_second_half; + std::vector ranges_first_half; + std::vector ranges_second_half; for (int i = 0; i < len; ++i) { int disk_id = i % num_disks; if (i > len / 2) { @@ -546,7 +546,7 @@ TEST_F(DiskIoMgrTest, SyncReadTest) { validate_sync_read(&io_mgr, reader, complete_range, data); validate_sync_read(&io_mgr, reader, complete_range, data); - vector ranges; + std::vector ranges; for (int i = 0; i < len; ++i) { int disk_id = i % num_disks; ranges.push_back(init_range(num_buffers, tmp_file, 0, len, disk_id, @@ -612,7 +612,7 @@ TEST_F(DiskIoMgrTest, SingleReaderCancel) { status = io_mgr.register_context(&reader, reader_mem_tracker); ASSERT_TRUE(status.ok()); - vector ranges; + std::vector ranges; for (int i = 0; i < len; ++i) { int disk_id = i % num_disks; ranges.push_back(init_range(num_buffers, tmp_file, 0, len, disk_id, @@ -682,7 +682,7 @@ TEST_F(DiskIoMgrTest, MemTrackers) { status = io_mgr.register_context(&reader, reader_mem_tracker); ASSERT_TRUE(status.ok()); - vector ranges; + std::vector ranges; for (int i = 0; i < num_buffers; ++i) { ranges.push_back(init_range(num_buffers, tmp_file, 0, len, 0, stat_val.st_mtime)); @@ -691,7 +691,7 @@ TEST_F(DiskIoMgrTest, MemTrackers) { ASSERT_TRUE(status.ok()); // Don't return buffers to force memory pressure - vector buffers; + std::vector buffers; AtomicInt num_ranges_processed; scan_range_thread(&io_mgr, reader, data, strlen(data), Status::MemoryLimitExceeded("Mem"), @@ -767,7 +767,7 @@ TEST_F(DiskIoMgrTest, CachedReads) { validate_sync_read(&io_mgr, reader, complete_range, data); validate_sync_read(&io_mgr, reader, complete_range, data); - vector ranges; + std::vector ranges; for (int i = 0; i < len; ++i) { int disk_id = i % num_disks; ranges.push_back(init_range(num_buffers, tmp_file, 0, len, disk_id, @@ -824,7 +824,7 @@ TEST_F(DiskIoMgrTest, MultipleReaderWriter) { stat(file_name.c_str(), &stat_val); int64_t iters = 0; - vector contexts(num_contexts); + std::vector contexts(num_contexts); Status status; for (int iteration = 0; iteration < ITERATIONS; ++iteration) { for (int threads_per_disk = 1; threads_per_disk <= 5; ++threads_per_disk) { @@ -845,7 +845,7 @@ TEST_F(DiskIoMgrTest, MultipleReaderWriter) { } AtomicInt num_ranges_processed; thread_group threads; - vector ranges; + std::vector ranges; int num_scan_ranges = std::min(num_reads_queued, write_offset - read_offset); for (int i = 0; i < num_scan_ranges; ++i) { ranges.push_back(init_range(1, file_name.c_str(), read_offset, 1, @@ -900,11 +900,11 @@ TEST_F(DiskIoMgrTest, MultipleReader) { const int ITERATIONS = 25; const int NUM_THREADS_PER_READER = 3; - vector file_names; - vector mtimes; - vector data; - vector readers; - vector results; + std::vector file_names; + std::vector mtimes; + std::vector data; + std::vector readers; + std::vector results; file_names.resize(NUM_READERS); readers.resize(NUM_READERS); @@ -922,7 +922,7 @@ TEST_F(DiskIoMgrTest, MultipleReader) { } data[i] = string(buf, DATA_LEN); - stringstream ss; + std::stringstream ss; ss << "/tmp/disk_io_mgr_test" << i << ".txt"; file_names[i] = ss.str(); CreateTempFile(ss.str().c_str(), data[i].c_str()); @@ -955,7 +955,7 @@ TEST_F(DiskIoMgrTest, MultipleReader) { status = io_mgr.register_context(&readers[i], NULL); ASSERT_TRUE(status.ok()); - vector ranges; + std::vector ranges; for (int j = 0; j < DATA_LEN; ++j) { int disk_id = j % num_disks; ranges.push_back( @@ -1067,7 +1067,7 @@ TEST_F(DiskIoMgrTest, PartialRead) { stat(tmp_file, &stat_val); _pool.reset(new ObjectPool); - scoped_ptr io_mgr(new DiskIoMgr(1, 1, read_len, read_len)); + boost::scoped_ptr io_mgr(new DiskIoMgr(1, 1, read_len, read_len)); Status status = io_mgr->init(mem_tracker); ASSERT_TRUE(status.ok()); diff --git a/be/test/runtime/memory_scratch_sink_test.cpp b/be/test/runtime/memory_scratch_sink_test.cpp index a7b2321a15..d607e7f40a 100644 --- a/be/test/runtime/memory_scratch_sink_test.cpp +++ b/be/test/runtime/memory_scratch_sink_test.cpp @@ -179,10 +179,10 @@ void MemoryScratchSinkTest::init_desc_tbl() { DescriptorTbl::create(&_obj_pool, _t_desc_table, &_desc_tbl); - vector row_tids; + std::vector row_tids; row_tids.push_back(0); - vector nullable_tuples; + std::vector nullable_tuples; nullable_tuples.push_back(false); _row_desc = _obj_pool.add(new RowDescriptor(*_desc_tbl, row_tids, nullable_tuples)); diff --git a/be/test/runtime/tmp_file_mgr_test.cpp b/be/test/runtime/tmp_file_mgr_test.cpp index 62fec4d9bf..1b4e7f7d90 100644 --- a/be/test/runtime/tmp_file_mgr_test.cpp +++ b/be/test/runtime/tmp_file_mgr_test.cpp @@ -42,7 +42,7 @@ protected: // Check that metric values are consistent with TmpFileMgr state. void check_metrics(TmpFileMgr* tmp_file_mgr) { - vector active = tmp_file_mgr->active_tmp_devices(); + std::vector active = tmp_file_mgr->active_tmp_devices(); int64_t active_metric = DorisMetrics::instance()->metric_registry()->get_entity("server")->get_metric("active_scratch_dirs").value(); EXPECT_EQ(active.size(), active_metric); @@ -56,7 +56,7 @@ TEST_F(TmpFileMgrTest, TestFileAllocation) { EXPECT_TRUE(tmp_file_mgr.init().ok()); // Default configuration should give us one temporary device. EXPECT_EQ(1, tmp_file_mgr.num_active_tmp_devices()); - vector tmp_devices = tmp_file_mgr.active_tmp_devices(); + std::vector tmp_devices = tmp_file_mgr.active_tmp_devices(); EXPECT_EQ(1, tmp_devices.size()); TUniqueId id; TmpFileMgr::File *file; @@ -86,7 +86,7 @@ TEST_F(TmpFileMgrTest, TestFileAllocation) { // Test that we can do initialization with two directories on same device and // that validations prevents duplication of directories. TEST_F(TmpFileMgrTest, TestOneDirPerDevice) { - vector tmp_dirs; + std::vector tmp_dirs; tmp_dirs.push_back("/tmp/tmp-file-mgr-test.1"); tmp_dirs.push_back("/tmp/tmp-file-mgr-test.2"); for (int i = 0; i < tmp_dirs.size(); ++i) { @@ -97,7 +97,7 @@ TEST_F(TmpFileMgrTest, TestOneDirPerDevice) { // Only the first directory should be used. EXPECT_EQ(1, tmp_file_mgr.num_active_tmp_devices()); - vector devices = tmp_file_mgr.active_tmp_devices(); + std::vector devices = tmp_file_mgr.active_tmp_devices(); EXPECT_EQ(1, devices.size()); TUniqueId id; TmpFileMgr::File *file; @@ -110,7 +110,7 @@ TEST_F(TmpFileMgrTest, TestOneDirPerDevice) { // Test that we can do custom initialization with two dirs on same device. TEST_F(TmpFileMgrTest, TestMultiDirsPerDevice) { - vector tmp_dirs; + std::vector tmp_dirs; tmp_dirs.push_back("/tmp/tmp-file-mgr-test.1"); tmp_dirs.push_back("/tmp/tmp-file-mgr-test.2"); for (int i = 0; i < tmp_dirs.size(); ++i) { @@ -121,7 +121,7 @@ TEST_F(TmpFileMgrTest, TestMultiDirsPerDevice) { // Both directories should be used. EXPECT_EQ(2, tmp_file_mgr.num_active_tmp_devices()); - vector devices = tmp_file_mgr.active_tmp_devices(); + std::vector devices = tmp_file_mgr.active_tmp_devices(); EXPECT_EQ(2, devices.size()); for (int i = 0; i < tmp_dirs.size(); ++i) { EXPECT_EQ(0, tmp_file_mgr.get_tmp_dir_path(devices[i]).find(tmp_dirs[i])); @@ -138,7 +138,7 @@ TEST_F(TmpFileMgrTest, TestMultiDirsPerDevice) { // Test that reporting a write error is possible but does not result in // blacklisting, which is disabled. TEST_F(TmpFileMgrTest, TestReportError) { - vector tmp_dirs; + std::vector tmp_dirs; tmp_dirs.push_back("/tmp/tmp-file-mgr-test.1"); tmp_dirs.push_back("/tmp/tmp-file-mgr-test.2"); for (int i = 0; i < tmp_dirs.size(); ++i) { @@ -148,7 +148,7 @@ TEST_F(TmpFileMgrTest, TestReportError) { tmp_file_mgr.init_custom(tmp_dirs, false); // Both directories should be used. - vector devices = tmp_file_mgr.active_tmp_devices(); + std::vector devices = tmp_file_mgr.active_tmp_devices(); EXPECT_EQ(2, devices.size()); // check_metrics(&tmp_file_mgr); @@ -166,7 +166,7 @@ TEST_F(TmpFileMgrTest, TestReportError) { EXPECT_FALSE(bad_file->is_blacklisted()); // The second device should still be active. EXPECT_EQ(2, tmp_file_mgr.num_active_tmp_devices()); - vector devices_after = tmp_file_mgr.active_tmp_devices(); + std::vector devices_after = tmp_file_mgr.active_tmp_devices(); EXPECT_EQ(2, devices_after.size()); // check_metrics(&tmp_file_mgr); @@ -188,7 +188,7 @@ TEST_F(TmpFileMgrTest, TestReportError) { TEST_F(TmpFileMgrTest, TestAllocateFails) { string tmp_dir("/tmp/tmp-file-mgr-test.1"); string scratch_subdir = tmp_dir + "/doris-scratch"; - vector tmp_dirs(1, tmp_dir); + std::vector tmp_dirs(1, tmp_dir); EXPECT_TRUE(FileSystemUtil::create_directory(tmp_dir).ok()); TmpFileMgr tmp_file_mgr; tmp_file_mgr.init_custom(tmp_dirs, false); diff --git a/be/test/util/arrow/arrow_work_flow_test.cpp b/be/test/util/arrow/arrow_work_flow_test.cpp index 05806f819a..7e665d0952 100644 --- a/be/test/util/arrow/arrow_work_flow_test.cpp +++ b/be/test/util/arrow/arrow_work_flow_test.cpp @@ -247,10 +247,10 @@ void ArrowWorkFlowTest::init_desc_tbl() { DescriptorTbl::create(&_obj_pool, _t_desc_table, &_desc_tbl); - vector row_tids; + std::vector row_tids; row_tids.push_back(0); - vector nullable_tuples; + std::vector nullable_tuples; nullable_tuples.push_back(false); // node diff --git a/be/test/util/decompress_test.cpp b/be/test/util/decompress_test.cpp index fd0c95737e..6d130b9e6e 100644 --- a/be/test/util/decompress_test.cpp +++ b/be/test/util/decompress_test.cpp @@ -46,8 +46,8 @@ protected: } void RunTest(THdfsCompression::type format) { - scoped_ptr compressor; - scoped_ptr decompressor; + boost::scoped_ptr compressor; + boost::scoped_ptr decompressor; MemPool* mem_pool = new MemPool; EXPECT_TRUE( diff --git a/be/test/util/internal_queue_test.cpp b/be/test/util/internal_queue_test.cpp index 6cb81768c0..a8942e283b 100644 --- a/be/test/util/internal_queue_test.cpp +++ b/be/test/util/internal_queue_test.cpp @@ -121,7 +121,7 @@ TEST(InternalQueue, TestBasic) { // Add all the nodes and then remove every other one. TEST(InternalQueue, TestRemove) { - vector nodes; + std::vector nodes; nodes.resize(100); InternalQueue queue; @@ -158,7 +158,7 @@ const int VALIDATE_INTERVAL = 10000; void ProducerThread( InternalQueue* queue, int num_inserts, - vector* nodes, + std::vector* nodes, AtomicInt* counter, bool* failed) { for (int i = 0; i < num_inserts && !*failed; ++i) { @@ -178,7 +178,7 @@ void ConsumerThread( InternalQueue* queue, int num_consumes, int delta, - vector* results, + std::vector* results, bool* failed) { // Dequeued nodes should be strictly increasing. int previous_value = -1; @@ -208,7 +208,7 @@ void ConsumerThread( } TEST(InternalQueue, TestClear) { - vector nodes; + std::vector nodes; nodes.resize(100); InternalQueue queue; queue.enqueue(&nodes[0]); @@ -227,10 +227,10 @@ TEST(InternalQueue, TestClear) { } TEST(InternalQueue, TestSingleProducerSingleConsumer) { - vector nodes; + std::vector nodes; AtomicInt counter; nodes.resize(1000000); - vector results; + std::vector results; InternalQueue queue; bool failed = false; @@ -252,7 +252,7 @@ TEST(InternalQueue, TestSingleProducerSingleConsumer) { } TEST(InternalQueue, TestMultiProducerMultiConsumer) { - vector nodes; + std::vector nodes; nodes.resize(1000000); bool failed = false; @@ -264,7 +264,7 @@ TEST(InternalQueue, TestMultiProducerMultiConsumer) { const int num_per_consumer = nodes.size() / NUM_CONSUMERS; const int num_per_producer = nodes.size() / num_producers; - vector> results; + std::vector> results; results.resize(NUM_CONSUMERS); int expected_delta = -1; @@ -301,7 +301,7 @@ TEST(InternalQueue, TestMultiProducerMultiConsumer) { ASSERT_TRUE(queue.empty()); ASSERT_TRUE(!failed); - vector all_results; + std::vector all_results; for (int i = 0; i < NUM_CONSUMERS; ++i) { ASSERT_EQ(results[i].size(), num_per_consumer); all_results.insert(all_results.end(), results[i].begin(), results[i].end()); diff --git a/be/test/util/path_util_test.cpp b/be/test/util/path_util_test.cpp index 62c1839d2c..132ea8052a 100644 --- a/be/test/util/path_util_test.cpp +++ b/be/test/util/path_util_test.cpp @@ -70,7 +70,7 @@ TEST(TestPathUtil, DirNameTest) { } TEST(TestPathUtil, SplitPathTest) { - using Vec = vector; + using Vec = std::vector; ASSERT_EQ(Vec({"/"}), path_util::split_path("/")); ASSERT_EQ(Vec({"/", "a", "b"}), path_util::split_path("/a/b")); ASSERT_EQ(Vec({"/", "a", "b"}), path_util::split_path("/a/b/")); diff --git a/be/test/util/perf_counters_test.cpp b/be/test/util/perf_counters_test.cpp index 92c0da2530..4ac307cd00 100644 --- a/be/test/util/perf_counters_test.cpp +++ b/be/test/util/perf_counters_test.cpp @@ -60,10 +60,10 @@ TEST(CpuInfoTest, Basic) { TEST(DiskInfoTest, Basic) { cout << DiskInfo::DebugString(); - cout << "Device name for disk 0: " << DiskInfo::device_name(0) << endl; + cout << "Device name for disk 0: " << DiskInfo::device_name(0) << std::endl; int disk_id_home_dir = DiskInfo::disk_id("/home"); - cout << "Device name for '/home': " << DiskInfo::device_name(disk_id_home_dir) << endl; + cout << "Device name for '/home': " << DiskInfo::device_name(disk_id_home_dir) << std::endl; } } diff --git a/be/test/util/rle_encoding_test.cpp b/be/test/util/rle_encoding_test.cpp index 8c8491ca44..458067b756 100644 --- a/be/test/util/rle_encoding_test.cpp +++ b/be/test/util/rle_encoding_test.cpp @@ -49,7 +49,7 @@ class TestRle : public testing::Test {}; // exactly 'expected_encoding'. // if expected_len is not -1, it will validate the encoded size is correct. template -void ValidateRle(const vector& values, int bit_width, +void ValidateRle(const std::vector& values, int bit_width, uint8_t* expected_encoding, int expected_len) { faststring buffer; RleEncoder encoder(&buffer, bit_width); @@ -82,7 +82,7 @@ void ValidateRle(const vector& values, int bit_width, TEST(Rle, SpecificSequences) { const int kTestLen = 1024; uint8_t expected_buffer[kTestLen]; - vector values; + std::vector values; // Test 50 0' followed by 50 1's values.resize(100); @@ -129,7 +129,7 @@ TEST(Rle, SpecificSequences) { // is used, otherwise alternating values are used. void TestRleValues(int bit_width, int num_vals, int value = -1) { const uint64_t mod = bit_width == 64 ? 1ULL : 1ULL << bit_width; - vector values; + std::vector values; for (uint64_t v = 0; v < num_vals; ++v) { values.push_back((value != -1) ? value : (bit_width == 64 ? v : (v % mod))); } @@ -157,7 +157,7 @@ public: // Tests all true/false values TEST_F(BitRle, AllSame) { const int kTestLen = 1024; - vector values; + std::vector values; for (int v = 0; v < 2; ++v) { values.clear(); @@ -172,7 +172,7 @@ TEST_F(BitRle, AllSame) { // Test that writes out a repeated group and then a literal // group but flush before finishing. TEST_F(BitRle, Flush) { - vector values; + std::vector values; for (int i = 0; i < 16; ++i) values.push_back(1); values.push_back(false); ValidateRle(values, 1, nullptr, -1); @@ -191,7 +191,7 @@ TEST_F(BitRle, RandomBools) { while (iters < n_iters) { srand(iters++); if (iters % 10000 == 0) LOG(ERROR) << "Seed: " << iters; - vector values; + std::vector values; bool parity = 0; for (int i = 0; i < 1000; ++i) { int group_size = rand() % 20 + 1; // NOLINT(*) @@ -214,7 +214,7 @@ TEST_F(BitRle, Random64Bit) { while (iters < n_iters) { srand(iters++); if (iters % 10000 == 0) LOG(ERROR) << "Seed: " << iters; - vector values; + std::vector values; for (int i = 0; i < 1000; ++i) { int group_size = rand() % 20 + 1; // NOLINT(*) uint64_t cur_value = (static_cast(rand()) << 32) + static_cast(rand()); @@ -233,7 +233,7 @@ TEST_F(BitRle, Random64Bit) { // Test a sequence of 1 0's, 2 1's, 3 0's. etc // e.g. 011000111100000 TEST_F(BitRle, RepeatedPattern) { - vector values; + std::vector values; const int min_run = 1; const int max_run = 32; @@ -344,10 +344,10 @@ TEST_F(TestRle, TestRoundTripRandomSequencesWithRuns) { // through the encode/decode sequence. for (int rep = 0; rep < 100; rep++) { faststring buf; - string string_rep; + std::string string_rep; int num_bits = GenerateRandomBitString(10, &buf, &string_rep); RleDecoder decoder(buf.data(), buf.size(), 1); - string roundtrip_str; + std::string roundtrip_str; int rem_to_read = num_bits; size_t run_len; bool val; diff --git a/be/test/util/runtime_profile_test.cpp b/be/test/util/runtime_profile_test.cpp index c06591e04a..0a8e494fd2 100644 --- a/be/test/util/runtime_profile_test.cpp +++ b/be/test/util/runtime_profile_test.cpp @@ -159,7 +159,7 @@ TEST(CountersTest, MergeAndUpdate) { ValidateCounter(merged_profile, "Parent 1 Only", 2); ValidateCounter(merged_profile, "Parent 2 Only", 5); - vector children; + std::vector children; merged_profile->GetChildren(&children); EXPECT_EQ(children.size(), 3); @@ -183,7 +183,7 @@ TEST(CountersTest, MergeAndUpdate) { } // make sure we can print - stringstream dummy; + std::stringstream dummy; merged_profile->PrettyPrint(&dummy); // Update profile2 w/ profile1 and validate @@ -330,7 +330,7 @@ TEST(CountersTest, BucketCounters) { unit_counter->Set(1L); // Create the bucket counters and start sampling - vector buckets; + std::vector buckets; profile.AddBucketingCounters("BucketCounters", "", unit_counter, 2, &buckets); // Wait two seconds. diff --git a/be/test/util/threadpool_test.cpp b/be/test/util/threadpool_test.cpp index f3ea9893b3..13c2bc0ba0 100644 --- a/be/test/util/threadpool_test.cpp +++ b/be/test/util/threadpool_test.cpp @@ -54,7 +54,7 @@ using std::atomic; using std::shared_ptr; using std::string; using std::thread; -using std::unique_ptr; + using std::vector; using strings::Substitute; @@ -83,7 +83,7 @@ public: } protected: - unique_ptr _pool; + std::unique_ptr _pool; }; TEST_F(ThreadPoolTest, TestNoTaskOpenClose) { @@ -194,8 +194,8 @@ TEST_F(ThreadPoolTest, TestThreadPoolWithNoMaxThreads) { ASSERT_EQ((kNumCPUs * 2), _pool->num_threads()); // submit tasks on two tokens. Only two threads should be created. - unique_ptr t1 = _pool->new_token(ThreadPool::ExecutionMode::SERIAL); - unique_ptr t2 = _pool->new_token(ThreadPool::ExecutionMode::SERIAL); + std::unique_ptr t1 = _pool->new_token(ThreadPool::ExecutionMode::SERIAL); + std::unique_ptr t2 = _pool->new_token(ThreadPool::ExecutionMode::SERIAL); for (int i = 0; i < kNumCPUs * 2; i++) { ThreadPoolToken* t = (i % 2 == 0) ? t1.get() : t2.get(); ASSERT_TRUE(t->submit(SlowTask::new_slow_task(&latch)).ok()); @@ -363,7 +363,7 @@ INSTANTIATE_TEST_CASE_P(Tokens, ThreadPoolTestTokenTypes, ThreadPool::ExecutionMode::CONCURRENT)); TEST_P(ThreadPoolTestTokenTypes, TestTokenSubmitAndWait) { - unique_ptr t = _pool->new_token(GetParam()); + std::unique_ptr t = _pool->new_token(GetParam()); int i = 0; Status status = t->submit_func([&]() { SleepFor(MonoDelta::FromMilliseconds(1)); @@ -375,7 +375,7 @@ TEST_P(ThreadPoolTestTokenTypes, TestTokenSubmitAndWait) { } TEST_F(ThreadPoolTest, TestTokenSubmitsProcessedSerially) { - unique_ptr t = _pool->new_token(ThreadPool::ExecutionMode::SERIAL); + std::unique_ptr t = _pool->new_token(ThreadPool::ExecutionMode::SERIAL); int32_t seed = static_cast(GetCurrentTimeMicros()); srand(seed); Random r(seed); @@ -398,7 +398,7 @@ TEST_P(ThreadPoolTestTokenTypes, TestTokenSubmitsProcessedConcurrently) { const int kNumTokens = 5; ASSERT_TRUE(rebuild_pool_with_builder(ThreadPoolBuilder(kDefaultPoolName) .set_max_threads(kNumTokens)).ok()); - vector> tokens; + std::vector> tokens; // A violation to the tested invariant would yield a deadlock, so let's set // up an alarm to bail us out. @@ -430,7 +430,7 @@ TEST_F(ThreadPoolTest, TestTokenSubmitsNonSequential) { alarm(0); // Disable alarm on test exit. }); shared_ptr b = std::make_shared(kNumSubmissions + 1); - unique_ptr t = _pool->new_token(ThreadPool::ExecutionMode::CONCURRENT); + std::unique_ptr t = _pool->new_token(ThreadPool::ExecutionMode::CONCURRENT); for (int i = 0; i < kNumSubmissions; i++) { ASSERT_TRUE(t->submit_func([b]() { b->wait(); @@ -445,8 +445,8 @@ TEST_P(ThreadPoolTestTokenTypes, TestTokenShutdown) { ASSERT_TRUE(rebuild_pool_with_builder(ThreadPoolBuilder(kDefaultPoolName) .set_max_threads(4)).ok()); - unique_ptr t1(_pool->new_token(GetParam())); - unique_ptr t2(_pool->new_token(GetParam())); + std::unique_ptr t1(_pool->new_token(GetParam())); + std::unique_ptr t2(_pool->new_token(GetParam())); CountDownLatch l1(1); CountDownLatch l2(1); @@ -489,7 +489,7 @@ TEST_P(ThreadPoolTestTokenTypes, TestTokenWaitForAll) { int32_t seed = static_cast(GetCurrentTimeMicros()); srand(seed); Random r(seed); - vector> tokens; + std::vector> tokens; for (int i = 0; i < kNumTokens; i++) { tokens.emplace_back(_pool->new_token(GetParam())); } @@ -522,7 +522,7 @@ TEST_F(ThreadPoolTest, TestFuzz) { int32_t seed = static_cast(GetCurrentTimeMicros()); srand(seed); Random r(seed); - vector> tokens; + std::vector> tokens; for (int i = 0; i < kNumOperations; i++) { // Operation distribution: @@ -605,7 +605,7 @@ TEST_P(ThreadPoolTestTokenTypes, TestTokenSubmissionsAdhereToMaxQueueSize) { .set_max_queue_size(1)).ok()); CountDownLatch latch(1); - unique_ptr t = _pool->new_token(GetParam()); + std::unique_ptr t = _pool->new_token(GetParam()); SCOPED_CLEANUP({ latch.count_down(); }); @@ -625,7 +625,7 @@ TEST_F(ThreadPoolTest, TestTokenConcurrency) { const int kWaitThreads = 2; const int kSubmitThreads = 8; - vector> tokens; + std::vector> tokens; int32_t seed = static_cast(GetCurrentTimeMicros()); srand(seed); Random rng(seed); @@ -658,7 +658,7 @@ TEST_F(ThreadPoolTest, TestTokenConcurrency) { atomic total_num_tokens_submitted(0); CountDownLatch latch(1); - vector threads; + std::vector threads; for (int i = 0; i < kCycleThreads; i++) { // Pick a token at random and replace it. @@ -737,13 +737,13 @@ TEST_F(ThreadPoolTest, TestTokenConcurrency) { t.join(); } - LOG(INFO) << Substitute("Tokens cycled ($0 threads): $1", + LOG(INFO) << strings::Substitute("Tokens cycled ($0 threads): $1", kCycleThreads, total_num_tokens_cycled.load()); - LOG(INFO) << Substitute("Tokens shutdown ($0 threads): $1", + LOG(INFO) << strings::Substitute("Tokens shutdown ($0 threads): $1", kShutdownThreads, total_num_tokens_shutdown.load()); - LOG(INFO) << Substitute("Tokens waited ($0 threads): $1", + LOG(INFO) << strings::Substitute("Tokens waited ($0 threads): $1", kWaitThreads, total_num_tokens_waited.load()); - LOG(INFO) << Substitute("Tokens submitted ($0 threads): $1", + LOG(INFO) << strings::Substitute("Tokens submitted ($0 threads): $1", kSubmitThreads, total_num_tokens_submitted.load()); } diff --git a/be/test/util/trace_test.cpp b/be/test/util/trace_test.cpp index 0d5a9540eb..c6295edb98 100644 --- a/be/test/util/trace_test.cpp +++ b/be/test/util/trace_test.cpp @@ -55,8 +55,8 @@ class TraceTest : public ::testing::Test { }; // Replace all digits in 's' with the character 'X'. -static string XOutDigits(const string& s) { - string ret; +static std::string XOutDigits(const string& s) { + std::string ret; ret.reserve(s.size()); for (char c : s) { if (isdigit(c)) { @@ -73,7 +73,7 @@ TEST_F(TraceTest, TestBasic) { TRACE_TO(t, "hello $0, $1", "world", 12345); TRACE_TO(t, "goodbye $0, $1", "cruel world", 54321); - string result = XOutDigits(t->DumpToString(Trace::NO_FLAGS)); + std::string result = XOutDigits(t->DumpToString(Trace::NO_FLAGS)); ASSERT_EQ("XXXX XX:XX:XX.XXXXXX trace_test.cpp:XX] hello world, XXXXX\n" "XXXX XX:XX:XX.XXXXXX trace_test.cpp:XX] goodbye cruel world, XXXXX\n", result); diff --git a/be/test/util/url_coding_test.cpp b/be/test/util/url_coding_test.cpp index 38456021be..9e0a4b0c81 100644 --- a/be/test/util/url_coding_test.cpp +++ b/be/test/util/url_coding_test.cpp @@ -29,7 +29,7 @@ namespace doris { void test_url(const string& input, const string& expected_encoded, bool hive_compat) { std::string intermediate; url_encode(input, &intermediate, hive_compat); - string output; + std::string output; if (!expected_encoded.empty()) { EXPECT_EQ(intermediate, expected_encoded); @@ -39,18 +39,18 @@ void test_url(const string& input, const string& expected_encoded, bool hive_com EXPECT_EQ(input, output); // Convert string to vector and try that also - vector input_vector; + std::vector input_vector; input_vector.resize(input.size()); memcpy(&input_vector[0], input.c_str(), input.size()); - string intermediate2; + std::string intermediate2; url_encode(input_vector, &intermediate2, hive_compat); EXPECT_EQ(intermediate, intermediate2); } void test_base64(const string& input, const string& expected_encoded) { - string intermediate; + std::string intermediate; Base64Encode(input, &intermediate); - string output; + std::string output; if (!expected_encoded.empty()) { EXPECT_EQ(intermediate, expected_encoded); @@ -60,10 +60,10 @@ void test_base64(const string& input, const string& expected_encoded) { EXPECT_EQ(input, output); // Convert string to vector and try that also - vector input_vector; + std::vector input_vector; input_vector.resize(input.size()); memcpy(&input_vector[0], input.c_str(), input.size()); - string intermediate2; + std::string intermediate2; Base64Encode(input_vector, &intermediate2); EXPECT_EQ(intermediate, intermediate2); } @@ -100,8 +100,8 @@ TEST(Base64Test, Basic) { } TEST(HtmlEscapingTest, Basic) { - string before = "&"; - stringstream after; + std::string before = "&"; + std::stringstream after; EscapeForHtml(before, &after); EXPECT_EQ(after.str(), "<html><body>&amp"); }